From 37c855315644c86e9dfe91c18a44a2b8816c5e9a Mon Sep 17 00:00:00 2001 From: wudi Date: Mon, 24 Nov 2025 20:26:45 +0800 Subject: [PATCH 01/27] add cdc client read data --- fs_brokers/cdc_client/pom.xml | 219 +++++ .../doris/cdcclient/CdcClientApplication.java | 40 + .../doris/cdcclient/common/JobManager.java | 127 +++ .../cdcclient/constants/LoadConstants.java | 30 + .../controller/ClientController.java | 116 +++ .../exception/CdcClientException.java | 29 + .../doris/cdcclient/model/FetchRecordReq.java | 77 ++ .../cdcclient/model/FinishSplitsReq.java | 50 ++ .../doris/cdcclient/model/JobConfig.java | 59 ++ .../cdcclient/model/rest/ResponseBody.java | 134 +++ .../model/rest/ResponseEntityBuilder.java | 81 ++ .../model/rest/RestApiStatusCode.java | 33 + .../cdcclient/mysql/utils/ConfigUtil.java | 121 +++ .../cdcclient/source/factory/DataSource.java | 22 + .../source/factory/SourceReaderFactory.java | 61 ++ .../source/reader/RecordWithMeta.java | 55 ++ .../cdcclient/source/reader/SourceReader.java | 51 ++ .../cdcclient/source/reader/SplitRecords.java | 60 ++ .../reader/mysql/MySqlSourceReader.java | 785 ++++++++++++++++++ .../source/serialize/DorisRecord.java | 69 ++ .../serialize/DorisRecordSerializer.java | 27 + .../source/serialize/JsonSerializer.java | 250 ++++++ .../source/split/AbstractSourceSplit.java | 20 + .../cdcclient/source/split/BinlogSplit.java | 28 + .../cdcclient/source/split/SnapshotSplit.java | 94 +++ .../cdcclient/source/split/SourceSplit.java | 3 + .../src/main/resources/application.properties | 2 + .../src/main/resources/log4j.properties | 23 + 28 files changed, 2666 insertions(+) create mode 100644 fs_brokers/cdc_client/pom.xml create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/JobManager.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/CdcClientException.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FetchRecordReq.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FinishSplitsReq.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseBody.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseEntityBuilder.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestApiStatusCode.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/mysql/utils/ConfigUtil.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/DataSource.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/RecordWithMeta.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecord.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecordSerializer.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/JsonSerializer.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SourceSplit.java create mode 100644 fs_brokers/cdc_client/src/main/resources/application.properties create mode 100644 fs_brokers/cdc_client/src/main/resources/log4j.properties diff --git a/fs_brokers/cdc_client/pom.xml b/fs_brokers/cdc_client/pom.xml new file mode 100644 index 00000000000000..649ea163f0cc99 --- /dev/null +++ b/fs_brokers/cdc_client/pom.xml @@ -0,0 +1,219 @@ + + 4.0.0 + + org.apache + apache + 29 + + cdc_client + 1.0.0 + Doris Cdc Client + https://doris.apache.org/ + + + Apache 2.0 License + https://www.apache.org/licenses/LICENSE-2.0.html + repo + + + + scm:git:https://git@github.com/apache/doris.git + scm:git:https://git@github.com/apache/doris.git + scm:git:https://git@github.com/apache/doris.git + HEAD + + + GitHub + https://github.com/apache/doris/issues + + + + Dev Mailing List + dev@doris.apache.org + dev-subscribe@doris.apache.org + dev-unsubscribe@doris.apache.org + + + Commits Mailing List + commits@doris.apache.org + commits-subscribe@doris.apache.org + commits-unsubscribe@doris.apache.org + + + + 3.5.7 + 1.20.3 + 1.7.36 + 1.2-SNAPSHOT + 2.43.0 + 8.14 + 2.17 + + + + + org.springframework.boot + spring-boot-starter-web + ${spring.version} + + + org.springframework.boot + spring-boot-starter-tomcat + + + org.springframework.boot + spring-boot-starter-logging + + + + + + + + + + + + + + + + + + + + + + + + + org.springframework.boot + spring-boot-starter-jetty + ${spring.version} + + + org.springframework.boot + spring-boot-starter-actuator + ${spring.version} + + + org.springframework.boot + spring-boot-starter-log4j2 + ${spring.version} + + + org.apache.flink + flink-connector-mysql-cdc + 3.2.1 + + + org.apache.flink + flink-clients + ${flink.version} + + + org.apache.flink + flink-connector-base + ${flink.version} + + + + org.apache.flink + flink-table-api-java-bridge + ${flink.version} + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + org.slf4j + slf4j-log4j12 + ${slf4j.version} + + + + + + cdc-client + + + org.springframework.boot + spring-boot-maven-plugin + 2.0.3.RELEASE + + + + repackage + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 17 + 17 + + + + com.diffplug.spotless + spotless-maven-plugin + ${spotless.version} + + + + 1.17.0 + + + + + + + + + spotless-check + validate + + check + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + ${maven-checkstyle-plugin.version} + + + com.puppycrawl.tools + checkstyle + + ${checkstyle.version} + + + + + validate + validate + + check + + + + + true + ../../fe/check/suppressions.xml + true + ../../fe/check/checkstyle.xml + true + true + + + + + \ No newline at end of file diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java new file mode 100644 index 00000000000000..73d24783388f9c --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.boot.SpringApplication; +import org.springframework.boot.autoconfigure.SpringBootApplication; +import org.springframework.boot.context.properties.EnableConfigurationProperties; +import org.springframework.boot.web.servlet.ServletComponentScan; +import org.springframework.boot.web.servlet.support.SpringBootServletInitializer; + +import java.util.Arrays; + +@SpringBootApplication +@EnableConfigurationProperties +@ServletComponentScan +public class CdcClientApplication extends SpringBootServletInitializer { + private static final Logger LOG = LoggerFactory.getLogger(CdcClientApplication.class); + + public static void main(String[] args) { + LOG.info("args: {}, log.path: {}", Arrays.asList(args), System.getProperty("log.path")); + SpringApplication.run(CdcClientApplication.class, args); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/JobManager.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/JobManager.java new file mode 100644 index 00000000000000..f30a0f42af9c9b --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/JobManager.java @@ -0,0 +1,127 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.common; + +import org.apache.doris.cdcclient.source.factory.DataSource; +import org.apache.doris.cdcclient.source.factory.SourceReaderFactory; +import org.apache.doris.cdcclient.source.reader.SourceReader; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +/** + * 负责管理以 jobId 为粒度的 SourceReader 及其配置,避免 Controller 手动维护状态。 + */ +public class JobManager { + private static volatile JobManager INSTANCE; + private final Map jobContexts; + + private JobManager() { + this.jobContexts = new ConcurrentHashMap<>(); + } + + public static JobManager getInstance() { + if (INSTANCE == null) { + synchronized (JobManager.class) { + if (INSTANCE == null) { + INSTANCE = new JobManager(); + } + } + } + return INSTANCE; + } + + + public SourceReader getOrCreateReader(Long jobId, DataSource dataSource) { + JobContext context = getOrCreateContext(jobId, dataSource); + return context.getOrCreateReader(dataSource); + } + + public SourceReader getReader(Long jobId) { + JobContext context = jobContexts.get(jobId); + return context == null ? null : context.reader; + } + + public void attachOptions(Long jobId, DataSource dataSource, Map options) { + if (jobId == null || options == null) { + return; + } + JobContext context = getOrCreateContext(jobId, dataSource); + context.setOptions(options); + } + + public Map getOptions(Long jobId) { + JobContext context = jobContexts.get(jobId); + return context == null ? null : context.options; + } + + public void close(Long jobId) { + JobContext context = jobContexts.remove(jobId); + if (context != null) { + context.close(); + } + } + + public void clearAll() { + jobContexts.keySet().forEach(this::close); + } + + private JobContext getOrCreateContext(Long jobId, DataSource dataSource) { + Objects.requireNonNull(jobId, "jobId"); + Objects.requireNonNull(dataSource, "dataSource"); + return jobContexts.computeIfAbsent(jobId, id -> new JobContext(id, dataSource)); + } + + private static final class JobContext { + private final long jobId; + private volatile SourceReader reader; + private volatile Map options; + private volatile DataSource dataSource; + + private JobContext(long jobId, DataSource dataSource) { + this.jobId = jobId; + this.dataSource = dataSource; + } + + private synchronized SourceReader getOrCreateReader(DataSource source) { + if (reader == null) { + reader = SourceReaderFactory.createSourceReader(source); + reader.initialize(); + dataSource = source; + } else if (dataSource != source) { + throw new IllegalStateException( + String.format( + "Job %d already bound to datasource %s, cannot switch to %s", + jobId, dataSource, source)); + } + return reader; + } + + private void setOptions(Map options) { + this.options = options; + } + + private void close() { + if (reader != null) { + reader.close(jobId); + reader = null; + } + } + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java new file mode 100644 index 00000000000000..4052166b99591c --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java @@ -0,0 +1,30 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.constants; + +public class LoadConstants { + public static final String DB_SOURCE_TYPE = "db_source_type"; + public static final String HOST = "host"; + public static final String PORT = "port"; + public static final String USERNAME = "username"; + public static final String PASSWORD = "password"; + public static final String DATABASE_NAME = "database"; + public static final String INCLUDE_TABLES_LIST = "include_tables"; + public static final String EXCLUDE_TABLES_LIST = "exclude_tables"; + public static final String INCLUDE_SCHEMA_CHANGES = "include_schema_changes"; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java new file mode 100644 index 00000000000000..f24e6bc14cb2b7 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -0,0 +1,116 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.controller; + +import org.apache.doris.cdcclient.common.JobManager; +import org.apache.doris.cdcclient.model.FetchRecordReq; +import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; +import org.apache.doris.cdcclient.source.factory.DataSource; +import org.apache.doris.cdcclient.source.reader.SourceReader; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.web.bind.annotation.PathVariable; +import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.RestController; + +import java.util.List; +import java.util.Locale; +import java.util.Map; + +@RestController +public class ClientController { + private static final Logger LOG = LoggerFactory.getLogger(ClientController.class); + + /** + * Fetch source splits for snapshot + * @param config + * @return + * @throws Exception + */ + @RequestMapping(path = "/api/fetchSplits", method = RequestMethod.POST) + public Object fetchSplits(@RequestBody JobConfig config) throws Exception { + try { + SourceReader reader = + prepareReader(config.getJobId(), config.getDataSource(), config.getConfig()); + List splits = reader.getSourceSplits(config); + return ResponseEntityBuilder.ok(splits); + } catch (IllegalArgumentException ex) { + LOG.error("Failed to fetch splits, jobId={}", config.getJobId(), ex); + return ResponseEntityBuilder.badRequest(ex.getMessage()); + } + } + + /** + * Fetch records from source reader + * @param recordReq + * @return + */ + @RequestMapping(path = "/api/fetchRecords", method = RequestMethod.POST) + public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { + try { + SourceReader reader = + prepareReader(recordReq.getJobId(), recordReq.getDataSource(), recordReq.getConfig()); + Object response = reader.read(recordReq); + return ResponseEntityBuilder.ok(response); + } catch (Exception ex) { + LOG.error("Failed fetch record, jobId={}", recordReq.getJobId(), ex); + return ResponseEntityBuilder.badRequest(ex.getMessage()); + } + } + + /** + * Write records to backend + * @param recordReq + * @return + */ + @RequestMapping(path = "/api/writeRecord", method = RequestMethod.POST) + public Object writeRecord(@RequestBody FetchRecordReq recordReq) { + // 拉取数据,写入到backend,这里先不实现 + return null; + } + + @RequestMapping(path = "/api/close/{jobId}", method = RequestMethod.POST) + public Object close(@PathVariable long jobId) { + JobManager manager = JobManager.getInstance(); + manager.close(jobId); + return ResponseEntityBuilder.ok(true); + } + + private DataSource resolveDataSource(String source) { + if (source == null || source.trim().isEmpty()) { + throw new IllegalArgumentException("Missing dataSource"); + } + try { + return DataSource.valueOf(source.trim().toUpperCase(Locale.ROOT)); + } catch (IllegalArgumentException ex) { + throw new IllegalArgumentException("Unsupported dataSource: " + source, ex); + } + } + + private SourceReader prepareReader( + Long jobId, String dataSource, Map config) { + DataSource parsed = resolveDataSource(dataSource); + JobManager manager = JobManager.getInstance(); + manager.attachOptions(jobId, parsed, config); + return manager.getOrCreateReader(jobId, parsed); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/CdcClientException.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/CdcClientException.java new file mode 100644 index 00000000000000..574cf4c73b7bcc --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/CdcClientException.java @@ -0,0 +1,29 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.exception; + +public class CdcClientException extends RuntimeException { + + public CdcClientException(String message) { + super(message); + } + + public CdcClientException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FetchRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FetchRecordReq.java new file mode 100644 index 00000000000000..df21b8841537a0 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FetchRecordReq.java @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model; + +import java.util.Map; + +public class FetchRecordReq { + private boolean reload; + private int fetchSize; + private long jobId; + private String dataSource; + private Map meta; + private Map config; + + public boolean isReload() { + return reload; + } + + public void setReload(boolean reload) { + this.reload = reload; + } + + public int getFetchSize() { + return fetchSize; + } + + public void setFetchSize(int fetchSize) { + this.fetchSize = fetchSize; + } + + public long getJobId() { + return jobId; + } + + public void setJobId(long jobId) { + this.jobId = jobId; + } + + public String getDataSource() { + return dataSource; + } + + public void setDataSource(String dataSource) { + this.dataSource = dataSource; + } + + public Map getMeta() { + return meta; + } + + public void setMeta(Map meta) { + this.meta = meta; + } + + public Map getConfig() { + return config; + } + + public void setConfig(Map config) { + this.config = config; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FinishSplitsReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FinishSplitsReq.java new file mode 100644 index 00000000000000..1d4e09ff8eb229 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FinishSplitsReq.java @@ -0,0 +1,50 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model; + +import java.util.Map; + +public class FinishSplitsReq { + + private String datasource; + + private Map> finishedOffsets; + + public FinishSplitsReq() {} + + public FinishSplitsReq(String datasource, Map> finishedOffsets) { + this.datasource = datasource; + this.finishedOffsets = finishedOffsets; + } + + public String getDatasource() { + return datasource; + } + + public void setDatasource(String datasource) { + this.datasource = datasource; + } + + public Map> getFinishedOffsets() { + return finishedOffsets; + } + + public void setFinishedOffsets(Map> finishedOffsets) { + this.finishedOffsets = finishedOffsets; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java new file mode 100644 index 00000000000000..312fa582cff88a --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model; + +import java.util.Map; + +public class JobConfig { + + private Long jobId; + private String dataSource; + private Map config; + + public JobConfig() {} + + public JobConfig(Long jobId, String dataSource, Map config) { + this.jobId = jobId; + this.dataSource = dataSource; + this.config = config; + } + + public Long getJobId() { + return jobId; + } + + public void setJobId(Long jobId) { + this.jobId = jobId; + } + + public String getDataSource() { + return dataSource; + } + + public void setDataSource(String dataSource) { + this.dataSource = dataSource; + } + + public Map getConfig() { + return config; + } + + public void setConfig(Map config) { + this.config = config; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseBody.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseBody.java new file mode 100644 index 00000000000000..96b37050b5fbb8 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseBody.java @@ -0,0 +1,134 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model.rest; + +import java.util.Objects; + +/** + * The response body of restful api. + * + *

The getter setter methods of all member variables need to be retained to ensure that Spring + * can perform json format conversion. + * + * @param type of data + */ +public class ResponseBody { + // Used to describe the error message. If there are no errors, it displays "OK" + private String msg; + // The user displays an error code. + // If there is no error, 0 is displayed. + // If there is an error, it is usually Doris's internal error code, not the HTTP standard error + // code. + // The HTTP standard error code should be reflected in the return value of the HTTP protocol. + private int code = RestApiStatusCode.OK.code; + // to save the response body + private T data; + // to save the number of records in response body. + // currently not used and always be 0. + private int count; + + public ResponseBody() {} + + public ResponseBody msg(String msg) { + this.msg = msg; + return this; + } + + public ResponseBody code(RestApiStatusCode code) { + this.code = code.code; + return this; + } + + public ResponseBody data(T data) { + this.data = data; + return this; + } + + public String getMsg() { + return msg; + } + + public void setMsg(String msg) { + this.msg = msg; + } + + public int getCode() { + return code; + } + + public void setCode(int code) { + this.code = code; + } + + public T getData() { + return data; + } + + public void setData(T data) { + this.data = data; + } + + public void setCount(int count) { + this.count = count; + } + + public int getCount() { + return count; + } + + public ResponseBody commonError(String msg) { + this.code = RestApiStatusCode.COMMON_ERROR.code; + this.msg = msg; + return this; + } + + @Override + public String toString() { + return "ResponseBody{" + + "msg='" + + msg + + '\'' + + ", code=" + + code + + ", data=" + + data + + ", count=" + + count + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ResponseBody that = (ResponseBody) o; + return code == that.code + && count == that.count + && Objects.equals(msg, that.msg) + && Objects.equals(data, that.data); + } + + @Override + public int hashCode() { + return Objects.hash(msg, code, data, count); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseEntityBuilder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseEntityBuilder.java new file mode 100644 index 00000000000000..38b5dd2aa0aece --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseEntityBuilder.java @@ -0,0 +1,81 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model.rest; + +import org.springframework.http.HttpStatus; +import org.springframework.http.ResponseEntity; + +/** + * A utility class for creating a ResponseEntity easier. All response will return with http code + * 200, and a internal code represent the real code. + */ +public class ResponseEntityBuilder { + + public static ResponseEntity badRequest(Object data) { + ResponseBody body = + new ResponseBody() + .code(RestApiStatusCode.BAD_REQUEST) + .msg("Bad Request") + .data(data); + return ResponseEntity.status(HttpStatus.OK).body(body); + } + + public static ResponseEntity okWithCommonError(String msg) { + ResponseBody body = + new ResponseBody().code(RestApiStatusCode.COMMON_ERROR).msg("Error").data(msg); + return ResponseEntity.status(HttpStatus.OK).body(body); + } + + public static ResponseEntity ok(Object data) { + ResponseBody body = new ResponseBody().code(RestApiStatusCode.OK).msg("success").data(data); + return ResponseEntity.status(HttpStatus.OK).body(body); + } + + public static ResponseEntity ok() { + ResponseBody body = new ResponseBody().code(RestApiStatusCode.OK).msg("success"); + return ResponseEntity.status(HttpStatus.OK).body(body); + } + + public static ResponseEntity okWithEmpty() { + return ResponseEntity.status(HttpStatus.OK).build(); + } + + public static ResponseEntity unauthorized(Object data) { + ResponseBody body = + new ResponseBody() + .code(RestApiStatusCode.UNAUTHORIZED) + .msg("Unauthorized") + .data(data); + return ResponseEntity.status(HttpStatus.OK).body(body); + } + + public static ResponseEntity internalError(Object data) { + ResponseBody body = + new ResponseBody() + .code(RestApiStatusCode.INTERNAL_SERVER_ERROR) + .msg("Internal Error") + .data(data); + return ResponseEntity.status(HttpStatus.OK).body(body); + } + + public static ResponseEntity notFound(Object data) { + ResponseBody body = + new ResponseBody().code(RestApiStatusCode.NOT_FOUND).msg("Not Found").data(data); + return ResponseEntity.status(HttpStatus.OK).body(body); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestApiStatusCode.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestApiStatusCode.java new file mode 100644 index 00000000000000..a5ea74587038b7 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestApiStatusCode.java @@ -0,0 +1,33 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model.rest; + +public enum RestApiStatusCode { + OK(0), + COMMON_ERROR(1), + UNAUTHORIZED(401), + BAD_REQUEST(403), + NOT_FOUND(404), + INTERNAL_SERVER_ERROR(500); + + public int code; + + RestApiStatusCode(int code) { + this.code = code; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/mysql/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/mysql/utils/ConfigUtil.java new file mode 100644 index 00000000000000..6c49e94aa55a64 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/mysql/utils/ConfigUtil.java @@ -0,0 +1,121 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.mysql.utils; + +import org.apache.doris.cdcclient.constants.LoadConstants; +import org.apache.doris.cdcclient.model.JobConfig; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory; +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetBuilder; +import org.apache.flink.cdc.connectors.mysql.table.StartupOptions; + +import java.util.Map; +import java.util.Properties; + +public class ConfigUtil { + + public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { + Map cdcConfig = config.getConfig(); + MySqlSourceConfigFactory configFactory = new MySqlSourceConfigFactory(); + configFactory.hostname(cdcConfig.get(LoadConstants.HOST)); + configFactory.port(Integer.valueOf(cdcConfig.get(LoadConstants.PORT))); + configFactory.username(cdcConfig.get(LoadConstants.USERNAME)); + configFactory.password(cdcConfig.get(LoadConstants.PASSWORD)); + + String databaseName = cdcConfig.get(LoadConstants.DATABASE_NAME); + configFactory.databaseList(databaseName); + configFactory.serverId(String.valueOf(Math.abs(config.getJobId().hashCode()))); + + configFactory.includeSchemaChanges( + Boolean.parseBoolean(cdcConfig.get(LoadConstants.INCLUDE_SCHEMA_CHANGES))); + + String includingTables = cdcConfig.getOrDefault(LoadConstants.INCLUDE_TABLES_LIST, ".*"); + String includingPattern = String.format("(%s)\\.(%s)", databaseName, includingTables); + String excludingTables = cdcConfig.get(LoadConstants.EXCLUDE_TABLES_LIST); + if (StringUtils.isEmpty(excludingTables)) { + configFactory.tableList(includingPattern); + } else { + String excludingPattern = + String.format("?!(%s\\.(%s))$", databaseName, excludingTables); + String tableList = String.format("(%s)(%s)", excludingPattern, includingPattern); + configFactory.tableList(tableList); + } + + // setting startMode + String startupMode = cdcConfig.get(MySqlSourceOptions.SCAN_STARTUP_MODE.key()); + if ("initial".equalsIgnoreCase(startupMode)) { + configFactory.startupOptions(StartupOptions.initial()); + } else if ("earliest-offset".equalsIgnoreCase(startupMode)) { + configFactory.startupOptions(StartupOptions.earliest()); + } else if ("latest-offset".equalsIgnoreCase(startupMode)) { + configFactory.startupOptions(StartupOptions.latest()); + } else if ("specific-offset".equalsIgnoreCase(startupMode)) { + BinlogOffsetBuilder offsetBuilder = BinlogOffset.builder(); + String file = cdcConfig.get(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_FILE.key()); + Long pos = + Long.valueOf( + cdcConfig.get( + MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_POS.key())); + if (file != null && pos != null) { + offsetBuilder.setBinlogFilePosition(file, pos); + } else { + offsetBuilder.setBinlogFilePosition("", 0); + } + + if (cdcConfig.containsKey( + MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_EVENTS.key())) { + long skipEvents = + Long.parseLong( + cdcConfig.getOrDefault( + MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_EVENTS + .key(), + "0")); + offsetBuilder.setSkipEvents(skipEvents); + } + if (cdcConfig.containsKey( + MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_ROWS.key())) { + long skipRows = + Long.parseLong( + cdcConfig.getOrDefault( + MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_ROWS + .key(), + "0")); + offsetBuilder.setSkipRows(skipRows); + } + configFactory.startupOptions(StartupOptions.specificOffset(offsetBuilder.build())); + } else if ("timestamp".equalsIgnoreCase(startupMode)) { + Long ts = + Long.parseLong( + cdcConfig.get(MySqlSourceOptions.SCAN_STARTUP_TIMESTAMP_MILLIS.key())); + configFactory.startupOptions(StartupOptions.timestamp(ts)); + } + + Properties jdbcProperteis = new Properties(); + jdbcProperteis.put("allowPublicKeyRetrieval", "true"); + jdbcProperteis.put("useSSL", "false"); + configFactory.jdbcProperties(jdbcProperteis); + + // for debug + // configFactory.splitSize(1); + return configFactory.createConfig(0); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/DataSource.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/DataSource.java new file mode 100644 index 00000000000000..20f1f94ecf9524 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/DataSource.java @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.factory; + +public enum DataSource { + MYSQL +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java new file mode 100644 index 00000000000000..5c26e9a5cd9b2a --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java @@ -0,0 +1,61 @@ +package org.apache.doris.cdcclient.source.factory; + +import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.cdcclient.source.reader.mysql.MySqlSourceReader; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; + +/** + * SourceReader 注册中心,支持在启动阶段按需扩展数据源。 + */ +public final class SourceReaderFactory { + private static final Logger LOG = LoggerFactory.getLogger(SourceReaderFactory.class); + private static final Map>> REGISTRY = + new ConcurrentHashMap<>(); + + static { + register(DataSource.MYSQL, MySqlSourceReader::new); + } + + private SourceReaderFactory() {} + + public static void register(DataSource source, Supplier> supplier) { + Objects.requireNonNull(source, "source"); + Objects.requireNonNull(supplier, "supplier"); + REGISTRY.put(source, supplier); + LOG.info("Registered SourceReader provider for {}", source); + } + + public static void unregister(DataSource source) { + if (source == null) { + return; + } + REGISTRY.remove(source); + LOG.info("Unregistered SourceReader provider for {}", source); + } + + public static boolean supports(DataSource source) { + return source != null && REGISTRY.containsKey(source); + } + + public static Set supportedDataSources() { + return Collections.unmodifiableSet(REGISTRY.keySet()); + } + + public static SourceReader createSourceReader(DataSource source) { + Supplier> supplier = REGISTRY.get(source); + if (supplier == null) { + throw new IllegalArgumentException( + "Unsupported SourceReader with datasource : " + source); + } + return supplier.get(); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/RecordWithMeta.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/RecordWithMeta.java new file mode 100644 index 00000000000000..9c5dd5c7360ef8 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/RecordWithMeta.java @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.reader; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class RecordWithMeta { + private Map meta; + private List records; + + public RecordWithMeta() { + this.meta = new HashMap<>(); + this.records = new ArrayList<>(); + } + + public RecordWithMeta(Map meta, List records) { + this.meta = meta; + this.records = records; + } + + public Map getMeta() { + return meta; + } + + public void setMeta(Map meta) { + this.meta = meta; + } + + public List getRecords() { + return records; + } + + public void setRecords(List records) { + this.records = records; + } + +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java new file mode 100644 index 00000000000000..acb7a9497f66cb --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -0,0 +1,51 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.reader; + +import org.apache.doris.cdcclient.model.FetchRecordReq; +import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.cdcclient.source.split.SourceSplit; + +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.util.List; + +public interface SourceReader { + + /** Initialization, called when the program starts */ + void initialize(); + + /** + * Divide the data to be read. For example: split mysql to chunks + * + * @return + */ + List getSourceSplits(JobConfig config) throws JsonProcessingException; + + /** + * Reading Data + * + * @param meta + * @return + * @throws Exception + */ + RecordWithMeta read(FetchRecordReq meta) throws Exception; + + /** Called when closing */ + void close(Long jobId); +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java new file mode 100644 index 00000000000000..d7c712e463d116 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java @@ -0,0 +1,60 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.reader; + +import org.apache.flink.cdc.connectors.mysql.source.split.SourceRecords; +import org.apache.kafka.connect.source.SourceRecord; + +import java.util.Iterator; + +public class SplitRecords { + private final String splitId; + private final SourceRecords records; + private Iterator iterator; + + public SplitRecords(String splitId, SourceRecords records) { + this.splitId = splitId; + this.records = records; + this.iterator = records.iterator(); + } + + public String getSplitId() { + return splitId; + } + + public SourceRecords getRecords() { + return records; + } + + public Iterator getIterator() { + return iterator; + } + + public void setIterator(Iterator iterator) { + this.iterator = iterator; + } + + public boolean isEmpty() { + return splitId == null || records == null || !records.iterator().hasNext(); + } + + @Override + public String toString() { + return "SplitRecords{" + "split=" + splitId + ", records=" + records + '}'; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java new file mode 100644 index 00000000000000..d864fe13243b74 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -0,0 +1,785 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.reader.mysql; + +import org.apache.doris.cdcclient.constants.LoadConstants; +import org.apache.doris.cdcclient.model.FetchRecordReq; +import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.cdcclient.mysql.utils.ConfigUtil; +import org.apache.doris.cdcclient.source.reader.SplitRecords; +import org.apache.doris.cdcclient.source.reader.RecordWithMeta; +import org.apache.doris.cdcclient.source.serialize.DorisRecordSerializer; +import org.apache.doris.cdcclient.source.serialize.JsonSerializer; +import org.apache.doris.cdcclient.source.split.AbstractSourceSplit; +import org.apache.doris.cdcclient.source.split.BinlogSplit; +import org.apache.doris.cdcclient.source.split.SnapshotSplit; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlPartition; +import io.debezium.document.Array; +import io.debezium.relational.Column; +import io.debezium.relational.TableId; +import io.debezium.relational.history.HistoryRecord; +import io.debezium.relational.history.TableChanges; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.cdc.common.utils.Preconditions; +import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; +import org.apache.flink.cdc.connectors.mysql.debezium.reader.BinlogSplitReader; +import org.apache.flink.cdc.connectors.mysql.debezium.reader.DebeziumReader; +import org.apache.flink.cdc.connectors.mysql.debezium.reader.SnapshotSplitReader; +import org.apache.flink.cdc.connectors.mysql.debezium.task.context.StatefulTaskContext; +import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; +import org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlSnapshotSplitAssigner; +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; +import org.apache.flink.cdc.connectors.mysql.source.split.FinishedSnapshotSplitInfo; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlBinlogSplit; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSnapshotSplit; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSnapshotSplitState; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplit; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplitState; +import org.apache.flink.cdc.connectors.mysql.source.split.SourceRecords; +import org.apache.flink.cdc.connectors.mysql.source.utils.ChunkUtils; +import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; +import org.apache.flink.cdc.connectors.mysql.source.utils.TableDiscoveryUtils; +import org.apache.flink.cdc.connectors.mysql.table.StartupMode; +import org.apache.flink.cdc.debezium.history.FlinkJsonTableChangeSerializer; +import org.apache.flink.table.types.logical.RowType; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +public class MySqlSourceReader implements org.apache.doris.cdcclient.source.reader.SourceReader { + private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceReader.class); + private static ObjectMapper objectMapper = new ObjectMapper(); + private static final String SPLIT_ID = "splitId"; + private static final String FINISH_SPLITS = "finishSplits"; + private static final String ASSIGNED_SPLITS = "assignedSplits"; + private static final String SNAPSHOT_TABLE = "snapshotTable"; + private static final String PURE_BINLOG_PHASE = "pureBinlogPhase"; + private static final FlinkJsonTableChangeSerializer TABLE_CHANGE_SERIALIZER = + new FlinkJsonTableChangeSerializer(); + private DorisRecordSerializer> serializer; + private Map jobContexts; + + public MySqlSourceReader() { + this.serializer = new JsonSerializer(); + this.jobContexts = new ConcurrentHashMap<>(); + } + + @Override + public void initialize() {} + + @Override + public List getSourceSplits(JobConfig config) + throws JsonProcessingException { + MySqlSourceConfig sourceConfig = getSourceConfig(config); + StartupMode startupMode = sourceConfig.getStartupOptions().startupMode; + List remainingSnapshotSplits = new ArrayList<>(); + MySqlBinlogSplit remainingBinlogSplit = null; + if (startupMode.equals(StartupMode.INITIAL)) { + String snapshotTable = config.getConfig().get(SNAPSHOT_TABLE); + remainingSnapshotSplits = + startSplitChunks(sourceConfig, snapshotTable, config.getConfig()); + } else { + remainingBinlogSplit = + new MySqlBinlogSplit( + BINLOG_SPLIT_ID, + sourceConfig.getStartupOptions().binlogOffset, + BinlogOffset.ofNonStopping(), + new ArrayList<>(), + new HashMap<>(), + 0); + } + List splits = new ArrayList<>(); + if (!remainingSnapshotSplits.isEmpty()) { + for (MySqlSnapshotSplit snapshotSplit : remainingSnapshotSplits) { + String splitId = snapshotSplit.splitId(); + String tableId = snapshotSplit.getTableId().identifier(); + String splitStart = + snapshotSplit.getSplitStart() == null + ? null + : objectMapper.writeValueAsString(snapshotSplit.getSplitStart()); + String splitEnd = + snapshotSplit.getSplitEnd() == null + ? null + : objectMapper.writeValueAsString(snapshotSplit.getSplitEnd()); + String splitKey = snapshotSplit.getSplitKeyType().getFieldNames().get(0); + SnapshotSplit split = + new SnapshotSplit(splitId, tableId, splitKey, splitStart, splitEnd, null); + splits.add(split); + } + } else { + BinlogOffset startingOffset = remainingBinlogSplit.getStartingOffset(); + BinlogSplit binlogSplit = + new BinlogSplit(remainingBinlogSplit.splitId(), startingOffset.getOffset()); + splits.add(binlogSplit); + } + return splits; + } + + /** + * 1. If the SplitRecords iterator has it, read the iterator directly. 2. If there is a + * binlogreader, poll it. 3. If there is none, resubmit split. 4. If reload is true, need to + * reset binlogSplitReader and submit split. + * + * @param fetchRecord + * @return + * @throws Exception + */ + @Override + public RecordWithMeta read(FetchRecordReq fetchRecord) throws Exception { + JobConfig jobConfig = + new JobConfig(fetchRecord.getJobId(), fetchRecord.getDataSource(), fetchRecord.getConfig()); + LOG.debug( + "Start read for jobId={}, dataSource={}, reload={}", + jobConfig.getJobId(), + jobConfig.getDataSource(), + fetchRecord.isReload()); + RecordWithMeta recordResponse = new RecordWithMeta(); + Map offsetMeta = fetchRecord.getMeta(); + if (offsetMeta.isEmpty()) { + throw new RuntimeException("miss meta offset"); + } + + JobRuntimeContext jobRuntimeContext = getJobRuntimeContext(jobConfig.getJobId()); + SplitProcessingContext processingContext = + prepareSplitProcessingContext(jobRuntimeContext, jobConfig, offsetMeta); + + if (fetchRecord.isReload() && processingContext.getSplit() == null) { + processingContext = + reloadSplitProcessingContext(jobRuntimeContext, jobConfig, offsetMeta); + } + + return drainSplitRecords( + fetchRecord, jobConfig, offsetMeta, jobRuntimeContext, processingContext, recordResponse); + } + + /** + * 根据当前上下文准备 Split 信息: + * - 若已有正在消费的 split,直接复用; + * - 否则根据 offset 构造新的 snapshot/binlog split 并启动 reader。 + */ + private SplitProcessingContext prepareSplitProcessingContext( + JobRuntimeContext jobRuntimeContext, JobConfig jobConfig, Map offsetMeta) + throws Exception { + SplitProcessingContext context = new SplitProcessingContext(); + SplitRecords currentSplitRecords = jobRuntimeContext.getCurrentSplitRecords(); + if (currentSplitRecords == null) { + DebeziumReader currentReader = + jobRuntimeContext.getCurrentReader(); + if (currentReader instanceof BinlogSplitReader) { + currentSplitRecords = pollSplitRecordsWithCurrentReader(currentReader); + } else if (currentReader == null) { + Tuple2 splitFlag = createMySqlSplit(offsetMeta, jobConfig); + context.setSplit(splitFlag.f0); + context.setPureBinlogPhase(splitFlag.f1); + context.setReadBinlog(!context.getSplit().isSnapshotSplit()); + LOG.info( + "Job {} submitted new split {}, readBinlog={}", + jobConfig.getJobId(), + context.getSplit().splitId(), + context.isReadBinlog()); + currentSplitRecords = pollSplitRecordsWithSplit(context.getSplit(), jobConfig); + } else { + throw new IllegalStateException( + String.format( + "Unsupported reader type %s for job %d", + currentReader.getClass().getName(), jobConfig.getJobId())); + } + jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); + } + context.setSplitRecords(currentSplitRecords); + return context; + } + + /** + * 当客户端请求 reload 时,重新构造 split 并重置 binlog reader。 + */ + private SplitProcessingContext reloadSplitProcessingContext( + JobRuntimeContext jobRuntimeContext, JobConfig jobConfig, Map offsetMeta) + throws Exception { + SplitProcessingContext context = new SplitProcessingContext(); + Tuple2 splitFlag = createMySqlSplit(offsetMeta, jobConfig); + context.setSplit(splitFlag.f0); + context.setPureBinlogPhase(splitFlag.f1); + context.setReadBinlog(!context.getSplit().isSnapshotSplit()); + LOG.info( + "Job {} reload split {}, readBinlog={}", + jobConfig.getJobId(), + context.getSplit() == null ? "null" : context.getSplit().splitId(), + context.isReadBinlog()); + closeBinlogReader(jobConfig.getJobId()); + SplitRecords currentSplitRecords = pollSplitRecordsWithSplit(context.getSplit(), jobConfig); + jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); + context.setSplitRecords(currentSplitRecords); + return context; + } + + /** + * 消费当前 split 中的记录并封装响应,同时更新元信息。 + */ + private RecordWithMeta drainSplitRecords( + FetchRecordReq fetchRecord, + JobConfig jobConfig, + Map offsetMeta, + JobRuntimeContext jobRuntimeContext, + SplitProcessingContext processingContext, + RecordWithMeta recordResponse) + throws Exception { + int count = 0; + int fetchSize = fetchRecord.getFetchSize(); + SplitRecords currentSplitRecords = processingContext.getSplitRecords(); + boolean readBinlog = processingContext.isReadBinlog(); + boolean pureBinlogPhase = processingContext.isPureBinlogPhase(); + MySqlSplit split = processingContext.getSplit(); + + MySqlSplitState currentSplitState = null; + if (!readBinlog && split != null) { + currentSplitState = new MySqlSnapshotSplitState(split.asSnapshotSplit()); + } + if (currentSplitRecords != null && !currentSplitRecords.isEmpty()) { + Iterator recordIt = currentSplitRecords.getIterator(); + while (recordIt.hasNext()) { + SourceRecord element = recordIt.next(); + if (RecordUtils.isWatermarkEvent(element)) { + BinlogOffset watermark = RecordUtils.getWatermark(element); + if (RecordUtils.isHighWatermarkEvent(element) && currentSplitState != null) { + currentSplitState.asSnapshotSplitState().setHighWatermark(watermark); + } + } else if (RecordUtils.isHeartbeatEvent(element)) { + LOG.debug("Receive heartbeat event: {}", element); + } else if (RecordUtils.isDataChangeRecord(element)) { + count += + handleDataChangeRecord( + jobConfig, + recordResponse, + element, + readBinlog, + pureBinlogPhase); + if (count >= fetchSize) { + return recordResponse; + } + } else { + LOG.debug("Ignore event: {}", element); + } + } + } + jobRuntimeContext.setCurrentSplitRecords(null); + if (!readBinlog && currentSplitState != null) { + BinlogOffset highWatermark = + currentSplitState.asSnapshotSplitState().getHighWatermark(); + Map offsetRes = highWatermark.getOffset(); + offsetRes.put(SPLIT_ID, split.splitId()); + recordResponse.setMeta(offsetRes); + } + if (CollectionUtils.isEmpty(recordResponse.getRecords())) { + if (readBinlog) { + Map offsetRes = new HashMap<>(offsetMeta); + if (split != null) { + offsetRes = split.asBinlogSplit().getStartingOffset().getOffset(); + } + offsetRes.put(SPLIT_ID, BINLOG_SPLIT_ID); + offsetRes.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + recordResponse.setMeta(offsetRes); + } else { + recordResponse.setMeta(fetchRecord.getMeta()); + } + } + return recordResponse; + } + + private int handleDataChangeRecord( + JobConfig jobConfig, + RecordWithMeta recordResponse, + SourceRecord element, + boolean readBinlog, + boolean pureBinlogPhase) + throws Exception { + List serialize = serializer.serialize(jobConfig.getConfig(), element); + if (CollectionUtils.isEmpty(serialize)) { + return 0; + } + LOG.debug( + "Job {} serialized {} records for split event", + jobConfig.getJobId(), + serialize.size()); + Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); + if (readBinlog) { + lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); + lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + recordResponse.setMeta(lastMeta); + } + recordResponse.getRecords().addAll(serialize); + return serialize.size(); + } + + /** + * refresh table changes after schema change + * + * @param element + * @param jobId + * @throws IOException + */ + private void refreshTableChanges(SourceRecord element, Long jobId) throws IOException { + HistoryRecord historyRecord = RecordUtils.getHistoryRecord(element); + Array tableChanges = historyRecord.document().getArray(HistoryRecord.Fields.TABLE_CHANGES); + TableChanges changes = TABLE_CHANGE_SERIALIZER.deserialize(tableChanges, true); + JobRuntimeContext jobContext = jobContexts.get(jobId); + if (jobContext == null) { + return; + } + Map tableChangeMap = jobContext.getTableSchemas(); + if (tableChangeMap == null) { + tableChangeMap = new ConcurrentHashMap<>(); + jobContext.setTableSchemas(tableChangeMap); + } + for (TableChanges.TableChange tblChange : changes) { + tableChangeMap.put(tblChange.getTable().id(), tblChange); + } + } + + private Tuple2 createMySqlSplit( + Map offset, JobConfig jobConfig) throws JsonProcessingException { + Tuple2 splitRes = null; + String splitId = offset.get(SPLIT_ID); + if (!BINLOG_SPLIT_ID.equals(splitId)) { + MySqlSnapshotSplit split = createSnapshotSplit(offset, jobConfig); + splitRes = Tuple2.of(split, false); + } else { + splitRes = createBinlogSplit(offset, jobConfig); + } + return splitRes; + } + + private MySqlSnapshotSplit createSnapshotSplit(Map offset, JobConfig jobConfig) + throws JsonProcessingException { + String splitId = offset.get(SPLIT_ID); + SnapshotSplit snapshotSplit = objectMapper.convertValue(offset, SnapshotSplit.class); + TableId tableId = TableId.parse(snapshotSplit.getTableId()); + Object[] splitStart = + snapshotSplit.getSplitStart() == null + ? null + : objectMapper.readValue(snapshotSplit.getSplitStart(), Object[].class); + Object[] splitEnd = + snapshotSplit.getSplitEnd() == null + ? null + : objectMapper.readValue(snapshotSplit.getSplitEnd(), Object[].class); + String splitKey = snapshotSplit.getSplitKey(); + Map tableSchemas = getTableSchemas(jobConfig); + TableChanges.TableChange tableChange = tableSchemas.get(tableId); + Column splitColumn = tableChange.getTable().columnWithName(splitKey); + RowType splitType = ChunkUtils.getChunkKeyColumnType(splitColumn); + MySqlSnapshotSplit split = + new MySqlSnapshotSplit( + tableId, splitId, splitType, splitStart, splitEnd, null, tableSchemas); + return split; + } + + private Tuple2 createBinlogSplit( + Map meta, JobConfig config) throws JsonProcessingException { + MySqlSourceConfig sourceConfig = getSourceConfig(config); + BinlogOffset offsetConfig = null; + if (sourceConfig.getStartupOptions() != null) { + offsetConfig = sourceConfig.getStartupOptions().binlogOffset; + } + + List finishedSnapshotSplitInfos = new ArrayList<>(); + BinlogOffset minOffsetFinishSplits = null; + BinlogOffset maxOffsetFinishSplits = null; + if (meta.containsKey(FINISH_SPLITS) && meta.containsKey(ASSIGNED_SPLITS)) { + // Construct binlogsplit based on the finished split and assigned split. + String finishSplitsOffset = meta.remove(FINISH_SPLITS); + String assignedSplits = meta.remove(ASSIGNED_SPLITS); + Map> splitFinishedOffsets = + objectMapper.readValue( + finishSplitsOffset, + new TypeReference>>() {}); + Map assignedSplitsMap = + objectMapper.readValue( + assignedSplits, new TypeReference>() {}); + List assignedSplitLists = + assignedSplitsMap.values().stream() + .sorted(Comparator.comparing(AbstractSourceSplit::getSplitId)) + .collect(Collectors.toList()); + + for (SnapshotSplit split : assignedSplitLists) { + // find the min binlog offset + Map offsetMap = splitFinishedOffsets.get(split.getSplitId()); + BinlogOffset binlogOffset = new BinlogOffset(offsetMap); + if (minOffsetFinishSplits == null || binlogOffset.isBefore(minOffsetFinishSplits)) { + minOffsetFinishSplits = binlogOffset; + } + if (maxOffsetFinishSplits == null || binlogOffset.isAfter(maxOffsetFinishSplits)) { + maxOffsetFinishSplits = binlogOffset; + } + Object[] splitStart = + split.getSplitStart() == null + ? null + : objectMapper.readValue(split.getSplitStart(), Object[].class); + Object[] splitEnd = + split.getSplitEnd() == null + ? null + : objectMapper.readValue(split.getSplitEnd(), Object[].class); + + finishedSnapshotSplitInfos.add( + new FinishedSnapshotSplitInfo( + TableId.parse(split.getTableId()), + split.getSplitId(), + splitStart, + splitEnd, + binlogOffset)); + } + } + + BinlogOffset startOffset; + BinlogOffset lastOffset = new BinlogOffset(meta); + if (minOffsetFinishSplits != null && lastOffset.getOffsetKind() == null) { + startOffset = minOffsetFinishSplits; + } else if (lastOffset.getOffsetKind() != null && lastOffset.getFilename() != null) { + startOffset = lastOffset; + } else if (offsetConfig != null) { + startOffset = offsetConfig; + } else { + startOffset = BinlogOffset.ofEarliest(); + } + + boolean pureBinlogPhase = false; + if (maxOffsetFinishSplits == null) { + pureBinlogPhase = true; + } else if (startOffset.isAtOrAfter(maxOffsetFinishSplits)) { + // All the offsets of the current split are smaller than the offset of the binlog, + // indicating that the binlog phase has been fully entered. + pureBinlogPhase = true; + LOG.info( + "The binlog phase has been fully entered, the current split is: {}", + startOffset); + } + + MySqlBinlogSplit split = + new MySqlBinlogSplit( + BINLOG_SPLIT_ID, + startOffset, + BinlogOffset.ofNonStopping(), + finishedSnapshotSplitInfos, + new HashMap<>(), + 0); + // filterTableSchema + MySqlBinlogSplit binlogSplit = + MySqlBinlogSplit.fillTableSchemas(split.asBinlogSplit(), getTableSchemas(config)); + return Tuple2.of(binlogSplit, pureBinlogPhase); + } + + private List startSplitChunks( + MySqlSourceConfig sourceConfig, String snapshotTable, Map config) { + List remainingTables = new ArrayList<>(); + if (snapshotTable != null) { + // need add database name + String database = config.get(LoadConstants.DATABASE_NAME); + remainingTables.add(TableId.parse(database + "." + snapshotTable)); + } + List remainingSplits = new ArrayList<>(); + MySqlSnapshotSplitAssigner splitAssigner = + new MySqlSnapshotSplitAssigner(sourceConfig, 1, remainingTables, false); + splitAssigner.open(); + while (true) { + Optional mySqlSplit = splitAssigner.getNext(); + if (mySqlSplit.isPresent()) { + MySqlSnapshotSplit snapshotSplit = mySqlSplit.get().asSnapshotSplit(); + remainingSplits.add(snapshotSplit); + } else { + break; + } + } + splitAssigner.close(); + return remainingSplits; + } + + private SplitRecords pollSplitRecordsWithSplit(MySqlSplit split, JobConfig jobConfig) + throws Exception { + Preconditions.checkState(split != null, "split is null"); + JobRuntimeContext jobContext = getJobRuntimeContext(jobConfig.getJobId()); + Iterator dataIt = null; + String currentSplitId = null; + DebeziumReader currentReader = null; + LOG.info("Get a split: {}", split.splitId()); + if (split instanceof MySqlSnapshotSplit) { + currentReader = getSnapshotSplitReader(jobConfig); + } else if (split instanceof MySqlBinlogSplit) { + currentReader = getBinlogSplitReader(jobConfig); + } + jobContext.setCurrentReader(currentReader); + currentReader.submitSplit(split); + currentSplitId = split.splitId(); + // make split record available + Thread.sleep(100); + dataIt = currentReader.pollSplitRecords(); + if (currentReader instanceof SnapshotSplitReader) { + closeSnapshotReader(jobConfig.getJobId()); + } + return dataIt == null ? null : new SplitRecords(currentSplitId, dataIt.next()); + } + + private SplitRecords pollSplitRecordsWithCurrentReader( + DebeziumReader currentReader) throws Exception { + Iterator dataIt = null; + if (currentReader instanceof BinlogSplitReader) { + dataIt = currentReader.pollSplitRecords(); + return dataIt == null ? null : new SplitRecords(BINLOG_SPLIT_ID, dataIt.next()); + } else { + throw new IllegalStateException("Unsupported reader type."); + } + } + + private SnapshotSplitReader getSnapshotSplitReader(JobConfig config) { + MySqlSourceConfig sourceConfig = getSourceConfig(config); + JobRuntimeContext jobContext = getJobRuntimeContext(config.getJobId()); + SnapshotSplitReader snapshotReader = jobContext.getSnapshotReader(); + if (snapshotReader == null) { + final MySqlConnection jdbcConnection = + DebeziumUtils.createMySqlConnection(sourceConfig); + final BinaryLogClient binaryLogClient = + DebeziumUtils.createBinaryClient(sourceConfig.getDbzConfiguration()); + final StatefulTaskContext statefulTaskContext = + new StatefulTaskContext(sourceConfig, binaryLogClient, jdbcConnection); + snapshotReader = new SnapshotSplitReader(statefulTaskContext, 0); + jobContext.setSnapshotReader(snapshotReader); + } + return snapshotReader; + } + + private BinlogSplitReader getBinlogSplitReader(JobConfig config) { + MySqlSourceConfig sourceConfig = getSourceConfig(config); + JobRuntimeContext jobContext = getJobRuntimeContext(config.getJobId()); + BinlogSplitReader binlogReader = jobContext.getBinlogReader(); + if (binlogReader == null) { + final MySqlConnection jdbcConnection = + DebeziumUtils.createMySqlConnection(sourceConfig); + final BinaryLogClient binaryLogClient = + DebeziumUtils.createBinaryClient(sourceConfig.getDbzConfiguration()); + final StatefulTaskContext statefulTaskContext = + new StatefulTaskContext(sourceConfig, binaryLogClient, jdbcConnection); + binlogReader = new BinlogSplitReader(statefulTaskContext, 0); + jobContext.setBinlogReader(binlogReader); + } + return binlogReader; + } + + private void closeSnapshotReader(Long jobId) { + JobRuntimeContext jobContext = jobContexts.get(jobId); + if (jobContext == null) { + return; + } + SnapshotSplitReader reusedSnapshotReader = jobContext.getSnapshotReader(); + if (reusedSnapshotReader != null) { + LOG.debug( + "Close snapshot reader {}", reusedSnapshotReader.getClass().getCanonicalName()); + reusedSnapshotReader.close(); + DebeziumReader currentReader = jobContext.getCurrentReader(); + if (reusedSnapshotReader == currentReader) { + jobContext.setCurrentReader(null); + } + jobContext.setSnapshotReader(null); + } + } + + private void closeBinlogReader(Long jobId) { + JobRuntimeContext jobContext = jobContexts.get(jobId); + if (jobContext == null) { + return; + } + BinlogSplitReader reusedBinlogReader = jobContext.getBinlogReader(); + if (reusedBinlogReader != null) { + LOG.debug("Close binlog reader {}", reusedBinlogReader.getClass().getCanonicalName()); + reusedBinlogReader.close(); + DebeziumReader currentReader = jobContext.getCurrentReader(); + if (reusedBinlogReader == currentReader) { + jobContext.setCurrentReader(null); + } + jobContext.setBinlogReader(null); + } + } + + private MySqlSourceConfig getSourceConfig(JobConfig config) { + return ConfigUtil.generateMySqlConfig(config); + } + + @Override + public void close(Long jobId) { + JobRuntimeContext jobContext = jobContexts.remove(jobId); + if (jobContext == null) { + return; + } + jobContext.close(); + LOG.info("Close source reader for job {}", jobId); + } + + private Map getTableSchemas(JobConfig config) { + JobRuntimeContext jobContext = getJobRuntimeContext(config.getJobId()); + Map schemas = jobContext.getTableSchemas(); + if (schemas == null) { + schemas = discoverTableSchemas(config); + jobContext.setTableSchemas(schemas); + } + return schemas; + } + + private Map discoverTableSchemas(JobConfig config) { + MySqlSourceConfig sourceConfig = getSourceConfig(config); + try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) { + MySqlPartition partition = + new MySqlPartition(sourceConfig.getMySqlConnectorConfig().getLogicalName()); + return TableDiscoveryUtils.discoverSchemaForCapturedTables( + partition, sourceConfig, jdbc); + } catch (SQLException ex) { + throw new RuntimeException(ex); + } + } + + private JobRuntimeContext getJobRuntimeContext(Long jobId) { + Objects.requireNonNull(jobId, "jobId"); + return jobContexts.computeIfAbsent(jobId, JobRuntimeContext::new); + } + + /** + * 临时保存一次 read 调用中与 split 相关的状态,避免大量方法参数。 + */ + private static final class SplitProcessingContext { + private SplitRecords splitRecords; + private MySqlSplit split; + private boolean readBinlog = true; + private boolean pureBinlogPhase = true; + + private SplitRecords getSplitRecords() { + return splitRecords; + } + + private void setSplitRecords(SplitRecords splitRecords) { + this.splitRecords = splitRecords; + } + + private MySqlSplit getSplit() { + return split; + } + + private void setSplit(MySqlSplit split) { + this.split = split; + } + + private boolean isReadBinlog() { + return readBinlog; + } + + private void setReadBinlog(boolean readBinlog) { + this.readBinlog = readBinlog; + } + + private boolean isPureBinlogPhase() { + return pureBinlogPhase; + } + + private void setPureBinlogPhase(boolean pureBinlogPhase) { + this.pureBinlogPhase = pureBinlogPhase; + } + } + + private static final class JobRuntimeContext { + private final long jobId; + private SnapshotSplitReader snapshotReader; + private BinlogSplitReader binlogReader; + private DebeziumReader currentReader; + private Map tableSchemas; + private SplitRecords currentSplitRecords; + + private JobRuntimeContext(Long jobId) { + this.jobId = jobId; + } + + private SnapshotSplitReader getSnapshotReader() { + return snapshotReader; + } + + private void setSnapshotReader(SnapshotSplitReader snapshotReader) { + this.snapshotReader = snapshotReader; + } + + private BinlogSplitReader getBinlogReader() { + return binlogReader; + } + + private void setBinlogReader(BinlogSplitReader binlogReader) { + this.binlogReader = binlogReader; + } + + private DebeziumReader getCurrentReader() { + return currentReader; + } + + private void setCurrentReader(DebeziumReader currentReader) { + this.currentReader = currentReader; + } + + private SplitRecords getCurrentSplitRecords() { + return currentSplitRecords; + } + + private void setCurrentSplitRecords(SplitRecords currentSplitRecords) { + this.currentSplitRecords = currentSplitRecords; + } + + private Map getTableSchemas() { + return tableSchemas; + } + + private void setTableSchemas(Map tableSchemas) { + this.tableSchemas = tableSchemas; + } + + private void close() { + if (snapshotReader != null) { + snapshotReader.close(); + snapshotReader = null; + } + if (binlogReader != null) { + binlogReader.close(); + binlogReader = null; + } + currentReader = null; + currentSplitRecords = null; + tableSchemas = null; + } + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecord.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecord.java new file mode 100644 index 00000000000000..25c4c343f24009 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecord.java @@ -0,0 +1,69 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.serialize; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; + +public class DorisRecord implements Serializable { + public static DorisRecord EOF = new DorisRecord(null, null, Arrays.asList("EOF")); + private String database; + private String table; + private List rows; + + public DorisRecord() {} + + public DorisRecord(String database, String table, List rows) { + this.database = database; + this.table = table; + this.rows = rows; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getTable() { + return table; + } + + public void setTable(String table) { + this.table = table; + } + + public List getRows() { + return rows; + } + + public void setRows(List rows) { + this.rows = rows; + } + + public void addRow(String row) { + this.rows.add(row); + } + + public String identifier() { + return database + "." + table; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecordSerializer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecordSerializer.java new file mode 100644 index 00000000000000..e501c4030b8f6c --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecordSerializer.java @@ -0,0 +1,27 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.serialize; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Map; + +public interface DorisRecordSerializer extends Serializable { + + C serialize(Map context, T record) throws IOException; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/JsonSerializer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/JsonSerializer.java new file mode 100644 index 00000000000000..35d83267c6cbb6 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/JsonSerializer.java @@ -0,0 +1,250 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.serialize; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.debezium.data.Envelope; +import io.debezium.data.SpecialValueDecimal; +import io.debezium.data.VariableScaleDecimal; +import io.debezium.relational.Tables; +import io.debezium.time.Date; +import io.debezium.time.MicroTimestamp; +import io.debezium.time.NanoTimestamp; +import io.debezium.time.Timestamp; +import io.debezium.time.ZonedTimestamp; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; +import org.apache.flink.cdc.debezium.utils.TemporalConversions; +import org.apache.flink.table.data.TimestampData; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class JsonSerializer implements DorisRecordSerializer> { + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(JsonSerializer.class); + private static final String DELETE_SIGN_KEY = "__DORIS_DELETE_SIGN__"; + private static ObjectMapper objectMapper = new ObjectMapper(); + private Map config; + private transient Tables tables; + + public JsonSerializer() {} + + @Override + public List serialize(Map context, SourceRecord record) + throws IOException { + if (RecordUtils.isDataChangeRecord(record)) { + LOG.trace("Process data change record: {}", record); + return deserializeDataChangeRecord(record); + } else if (RecordUtils.isSchemaChangeEvent(record)) { + LOG.debug("Ignored schema change record: {}", record); + return Collections.emptyList(); + } else { + LOG.trace("Ignored other record: {}", record); + return Collections.emptyList(); + } + } + + private List deserializeDataChangeRecord(SourceRecord record) throws IOException { + List rows = new ArrayList<>(); + Envelope.Operation op = Envelope.operationFor(record); + Struct value = (Struct) record.value(); + String table = value.getStruct(Envelope.FieldName.SOURCE).getString("table"); + Schema valueSchema = record.valueSchema(); + if (Envelope.Operation.DELETE.equals(op)) { + String deleteRow = extractBeforeRow(value, valueSchema); + if (StringUtils.isNotEmpty(deleteRow)) { + rows.add(table + "|" + deleteRow); + } + } else if (Envelope.Operation.READ.equals(op) + || Envelope.Operation.CREATE.equals(op) + || Envelope.Operation.UPDATE.equals(op)) { + String insertRow = extractAfterRow(value, valueSchema); + if (StringUtils.isNotEmpty(insertRow)) { + rows.add(table + "|" + insertRow); + } + } + return rows; + } + + private String extractAfterRow(Struct value, Schema valueSchema) + throws JsonProcessingException { + Map record = new HashMap<>(); + Struct after = value.getStruct(Envelope.FieldName.AFTER); + if (after == null) { + return null; + } + Schema afterSchema = valueSchema.field(Envelope.FieldName.AFTER).schema(); + afterSchema + .fields() + .forEach( + field -> { + Object valueConverted = + convert(field.schema(), after.getWithoutDefault(field.name())); + record.put(field.name(), valueConverted); + }); + record.put(DELETE_SIGN_KEY, 0); + System.out.println(record); + return objectMapper.writeValueAsString(record); + } + + private String extractBeforeRow(Struct value, Schema valueSchema) + throws JsonProcessingException { + Map record = new HashMap<>(); + Struct before = value.getStruct(Envelope.FieldName.BEFORE); + if (before == null) { + return null; + } + Schema beforeSchema = valueSchema.field(Envelope.FieldName.BEFORE).schema(); + beforeSchema + .fields() + .forEach( + field -> { + Object valueConverted = + convert(field.schema(), before.getWithoutDefault(field.name())); + record.put(field.name(), valueConverted); + }); + record.put(DELETE_SIGN_KEY, 1); + return objectMapper.writeValueAsString(record); + } + + private Object convert(Schema fieldSchema, Object dbzObj) { + if (dbzObj == null) { + return null; + } + String name = fieldSchema.name(); + Schema.Type type = fieldSchema.type(); + if (StringUtils.isEmpty(name)) { + switch (type) { + case INT8: + case INT16: + case INT32: + case INT64: + return Long.parseLong(dbzObj.toString()); + case FLOAT32: + return Float.parseFloat(dbzObj.toString()); + case FLOAT64: + return Double.parseDouble(dbzObj.toString()); + case BOOLEAN: + return Boolean.parseBoolean(dbzObj.toString()); + case STRING: + case ARRAY: + case MAP: + case STRUCT: + case BYTES: + return dbzObj.toString(); + default: + LOG.warn("Unsupported type: {}, transform value to string", type); + return dbzObj.toString(); + } + } else { + switch (name) { + case Date.SCHEMA_NAME: + return TemporalConversions.toLocalDate(dbzObj).toString(); + case Timestamp.SCHEMA_NAME: + case MicroTimestamp.SCHEMA_NAME: + case NanoTimestamp.SCHEMA_NAME: + return convertTimestamp(name, dbzObj); + case ZonedTimestamp.SCHEMA_NAME: + return convertZoneTimestamp(dbzObj); + case Decimal.LOGICAL_NAME: + return convertDecimal(dbzObj, fieldSchema); + default: + LOG.warn( + "Unsupported type: {} with name {}, transform value to string", + type, + name); + return dbzObj.toString(); + } + } + } + + private Object convertZoneTimestamp(Object dbzObj) { + if (dbzObj instanceof String) { + String str = (String) dbzObj; + // TIMESTAMP_LTZ type is encoded in string type + Instant instant = Instant.parse(str); + return TimestampData.fromLocalDateTime( + LocalDateTime.ofInstant(instant, ZoneId.systemDefault())) + .toTimestamp() + .toString(); + } + LOG.warn("Unable to convert to zone timestamp, default {}", dbzObj); + return dbzObj.toString(); + } + + private Object convertTimestamp(String typeName, Object dbzObj) { + if (dbzObj instanceof Long) { + switch (typeName) { + case Timestamp.SCHEMA_NAME: + return TimestampData.fromEpochMillis((Long) dbzObj).toTimestamp().toString(); + case MicroTimestamp.SCHEMA_NAME: + long micro = (long) dbzObj; + return TimestampData.fromEpochMillis(micro / 1000, (int) (micro % 1000 * 1000)) + .toTimestamp() + .toString(); + case NanoTimestamp.SCHEMA_NAME: + long nano = (long) dbzObj; + return TimestampData.fromEpochMillis(nano / 1000_000, (int) (nano % 1000_000)) + .toTimestamp() + .toString(); + } + } + LocalDateTime localDateTime = + TemporalConversions.toLocalDateTime(dbzObj, ZoneId.systemDefault()); + return java.sql.Timestamp.valueOf(localDateTime); + } + + private Object convertDecimal(Object dbzObj, Schema schema) { + BigDecimal bigDecimal; + if (dbzObj instanceof byte[]) { + // decimal.handling.mode=precise + bigDecimal = Decimal.toLogical(schema, (byte[]) dbzObj); + } else if (dbzObj instanceof String) { + // decimal.handling.mode=string + bigDecimal = new BigDecimal((String) dbzObj); + } else if (dbzObj instanceof Double) { + // decimal.handling.mode=double + bigDecimal = BigDecimal.valueOf((Double) dbzObj); + } else { + if (VariableScaleDecimal.LOGICAL_NAME.equals(schema.name())) { + SpecialValueDecimal decimal = VariableScaleDecimal.toLogical((Struct) dbzObj); + bigDecimal = decimal.getDecimalValue().orElse(BigDecimal.ZERO); + } else { + // fallback to string + bigDecimal = new BigDecimal(dbzObj.toString()); + } + } + return bigDecimal; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java new file mode 100644 index 00000000000000..cd8b85872d1a66 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java @@ -0,0 +1,20 @@ +package org.apache.doris.cdcclient.source.split; + +public abstract class AbstractSourceSplit implements SourceSplit { + private static final long serialVersionUID = 1L; + protected String splitId; + + public AbstractSourceSplit() {} + + public AbstractSourceSplit(String splitId) { + this.splitId = splitId; + } + + public String getSplitId() { + return splitId; + } + + public void setSplitId(String splitId) { + this.splitId = splitId; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java new file mode 100644 index 00000000000000..a7466feb199e46 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java @@ -0,0 +1,28 @@ +package org.apache.doris.cdcclient.source.split; + +import java.util.Map; + +public class BinlogSplit extends AbstractSourceSplit { + private static final long serialVersionUID = 1L; + private Map offset; + + public BinlogSplit() {} + + public BinlogSplit(String splitId, Map offset) { + super(splitId); + this.offset = offset; + } + + public Map getOffset() { + return offset; + } + + public void setOffset(Map offset) { + this.offset = offset; + } + + @Override + public String toString() { + return "BinlogSplit{" + "offset=" + offset + ", splitId='" + splitId + '\'' + '}'; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java new file mode 100644 index 00000000000000..92fe48fc73f607 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java @@ -0,0 +1,94 @@ +package org.apache.doris.cdcclient.source.split; + +import java.util.Map; + +public class SnapshotSplit extends AbstractSourceSplit { + private static final long serialVersionUID = 1L; + private String tableId; + private String splitKey; + private String splitStart; + private String splitEnd; + private Map highWatermark; + + public SnapshotSplit() { + super(); + } + + public SnapshotSplit( + String splitId, + String tableId, + String splitKey, + String splitStart, + String splitEnd, + Map highWatermark) { + super(splitId); + this.tableId = tableId; + this.splitKey = splitKey; + this.splitStart = splitStart; + this.splitEnd = splitEnd; + this.highWatermark = highWatermark; + } + + public String getTableId() { + return tableId; + } + + public void setTableId(String tableId) { + this.tableId = tableId; + } + + public String getSplitKey() { + return splitKey; + } + + public void setSplitKey(String splitKey) { + this.splitKey = splitKey; + } + + public String getSplitStart() { + return splitStart; + } + + public void setSplitStart(String splitStart) { + this.splitStart = splitStart; + } + + public String getSplitEnd() { + return splitEnd; + } + + public void setSplitEnd(String splitEnd) { + this.splitEnd = splitEnd; + } + + public Map getHighWatermark() { + return highWatermark; + } + + public void setHighWatermark(Map highWatermark) { + this.highWatermark = highWatermark; + } + + @Override + public String toString() { + return "SnapshotSplit{" + + "tableId='" + + tableId + + '\'' + + ", splitKey='" + + splitKey + + '\'' + + ", splitStart='" + + splitStart + + '\'' + + ", splitEnd='" + + splitEnd + + '\'' + + ", highWatermark=" + + highWatermark + + ", splitId='" + + splitId + + '\'' + + '}'; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SourceSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SourceSplit.java new file mode 100644 index 00000000000000..70c8be085ea28a --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SourceSplit.java @@ -0,0 +1,3 @@ +package org.apache.doris.cdcclient.source.split; + +public interface SourceSplit {} diff --git a/fs_brokers/cdc_client/src/main/resources/application.properties b/fs_brokers/cdc_client/src/main/resources/application.properties new file mode 100644 index 00000000000000..95a974099af040 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/resources/application.properties @@ -0,0 +1,2 @@ +spring.web.resources.add-mappings=false +server.port=9096 \ No newline at end of file diff --git a/fs_brokers/cdc_client/src/main/resources/log4j.properties b/fs_brokers/cdc_client/src/main/resources/log4j.properties new file mode 100644 index 00000000000000..ecb73d3811cf63 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/resources/log4j.properties @@ -0,0 +1,23 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +log4j.rootLogger=INFO, console + +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c [%t] %x - %m%n \ No newline at end of file From cd4e1e90c61220f1c9424d49a47e94601241b630 Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 25 Nov 2025 20:14:56 +0800 Subject: [PATCH 02/27] add a part of stream load sink --- fs_brokers/cdc_client/pom.xml | 48 +- .../doris/cdcclient/CdcClientApplication.java | 5 +- .../common/{JobManager.java => Env.java} | 77 ++- .../cdcclient/constants/LoadConstants.java | 1 - .../controller/ClientController.java | 80 ++- .../StreamLoadException.java} | 38 +- .../doris/cdcclient/model/JobConfig.java | 39 +- .../BaseRecordReq.java} | 55 +- .../cdcclient/model/req/FetchRecordReq.java | 38 ++ .../cdcclient/model/req/WriteRecordReq.java | 25 + .../reader => model/resp}/RecordWithMeta.java | 26 +- .../cdcclient/model/resp/WriteMetaResp.java | 9 + .../service/PipelineCoordinator.java | 181 +++++++ .../cdcclient/sink/BatchBufferHttpEntity.java | 75 +++ .../cdcclient/sink/BatchBufferStream.java | 73 +++ .../cdcclient/sink/BatchRecordBuffer.java | 155 ++++++ .../cdcclient/sink/DorisBatchStreamLoad.java | 482 ++++++++++++++++++ .../doris/cdcclient/sink/RespContent.java | 169 ++++++ .../DebeziumJsonDeserializer.java} | 19 +- .../SourceRecordDeserializer.java} | 6 +- .../source/factory/SourceReaderFactory.java | 32 +- .../cdcclient/source/reader/SourceReader.java | 30 +- .../source/reader/SplitReadResult.java | 46 ++ .../cdcclient/source/reader/SplitRecords.java | 3 +- .../reader/mysql/MySqlSourceReader.java | 318 ++++++++---- .../source/serialize/DorisRecord.java | 69 --- .../{mysql => }/utils/ConfigUtil.java | 5 +- .../doris/cdcclient/utils/HttpUtil.java | 38 ++ 28 files changed, 1684 insertions(+), 458 deletions(-) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/{JobManager.java => Env.java} (66%) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/{model/FinishSplitsReq.java => exception/StreamLoadException.java} (51%) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/{FetchRecordReq.java => req/BaseRecordReq.java} (51%) create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/FetchRecordReq.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/WriteRecordReq.java rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/{source/reader => model/resp}/RecordWithMeta.java (68%) create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/WriteMetaResp.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferHttpEntity.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferStream.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/RespContent.java rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/{serialize/JsonSerializer.java => deserialize/DebeziumJsonDeserializer.java} (93%) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/{serialize/DorisRecordSerializer.java => deserialize/SourceRecordDeserializer.java} (81%) create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java delete mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecord.java rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/{mysql => }/utils/ConfigUtil.java (96%) create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/HttpUtil.java diff --git a/fs_brokers/cdc_client/pom.xml b/fs_brokers/cdc_client/pom.xml index 649ea163f0cc99..7688139bd65b95 100644 --- a/fs_brokers/cdc_client/pom.xml +++ b/fs_brokers/cdc_client/pom.xml @@ -49,6 +49,9 @@ 2.43.0 8.14 2.17 + 4.5.13 + 1.18.24 + 1.15 @@ -124,16 +127,41 @@ ${flink.version} - - org.slf4j - slf4j-api - ${slf4j.version} - - - org.slf4j - slf4j-log4j12 - ${slf4j.version} - + + org.apache.httpcomponents + httpclient + ${httpcomponents.version} + + + commons-codec + commons-codec + + + + + + commons-codec + commons-codec + ${commons-codec.version} + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + org.slf4j + slf4j-log4j12 + ${slf4j.version} + + + + org.projectlombok + lombok + ${lombok.version} + provided + diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java index 73d24783388f9c..1f62cf1f56527d 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java @@ -17,6 +17,8 @@ package org.apache.doris.cdcclient; +import java.util.Arrays; +import org.apache.doris.cdcclient.common.Env; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.boot.SpringApplication; @@ -25,8 +27,6 @@ import org.springframework.boot.web.servlet.ServletComponentScan; import org.springframework.boot.web.servlet.support.SpringBootServletInitializer; -import java.util.Arrays; - @SpringBootApplication @EnableConfigurationProperties @ServletComponentScan @@ -35,6 +35,7 @@ public class CdcClientApplication extends SpringBootServletInitializer { public static void main(String[] args) { LOG.info("args: {}, log.path: {}", Arrays.asList(args), System.getProperty("log.path")); + Env.getCurrentEnv().setBeHttpPort(8040); SpringApplication.run(CdcClientApplication.class, args); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/JobManager.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java similarity index 66% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/JobManager.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java index f30a0f42af9c9b..147c3045b93c1c 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/JobManager.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java @@ -17,58 +17,57 @@ package org.apache.doris.cdcclient.common; -import org.apache.doris.cdcclient.source.factory.DataSource; -import org.apache.doris.cdcclient.source.factory.SourceReaderFactory; -import org.apache.doris.cdcclient.source.reader.SourceReader; - +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; +import lombok.Getter; +import lombok.Setter; +import org.apache.doris.cdcclient.source.factory.DataSource; +import org.apache.doris.cdcclient.source.factory.SourceReaderFactory; +import org.apache.doris.cdcclient.source.reader.SourceReader; -/** - * 负责管理以 jobId 为粒度的 SourceReader 及其配置,避免 Controller 手动维护状态。 - */ -public class JobManager { - private static volatile JobManager INSTANCE; +public class Env { + private static volatile Env INSTANCE; private final Map jobContexts; + @Getter @Setter private int beHttpPort; - private JobManager() { + private Env() { this.jobContexts = new ConcurrentHashMap<>(); } - public static JobManager getInstance() { + public static Env getCurrentEnv() { if (INSTANCE == null) { - synchronized (JobManager.class) { + synchronized (Env.class) { if (INSTANCE == null) { - INSTANCE = new JobManager(); + INSTANCE = new Env(); } } } return INSTANCE; } - - public SourceReader getOrCreateReader(Long jobId, DataSource dataSource) { - JobContext context = getOrCreateContext(jobId, dataSource); - return context.getOrCreateReader(dataSource); + public SourceReader getReader(Long jobId, String dataSource, Map config) { + DataSource ds = resolveDataSource(dataSource); + Env manager = Env.getCurrentEnv(); + return manager.getOrCreateReader(jobId, ds, config); } - public SourceReader getReader(Long jobId) { - JobContext context = jobContexts.get(jobId); - return context == null ? null : context.reader; - } - - public void attachOptions(Long jobId, DataSource dataSource, Map options) { - if (jobId == null || options == null) { - return; + private DataSource resolveDataSource(String source) { + if (source == null || source.trim().isEmpty()) { + throw new IllegalArgumentException("Missing dataSource"); + } + try { + return DataSource.valueOf(source.trim().toUpperCase(Locale.ROOT)); + } catch (IllegalArgumentException ex) { + throw new IllegalArgumentException("Unsupported dataSource: " + source, ex); } - JobContext context = getOrCreateContext(jobId, dataSource); - context.setOptions(options); } - public Map getOptions(Long jobId) { - JobContext context = jobContexts.get(jobId); - return context == null ? null : context.options; + private SourceReader getOrCreateReader( + Long jobId, DataSource dataSource, Map config) { + JobContext context = getOrCreateContext(jobId, dataSource, config); + return context.getOrCreateReader(dataSource); } public void close(Long jobId) { @@ -78,25 +77,23 @@ public void close(Long jobId) { } } - public void clearAll() { - jobContexts.keySet().forEach(this::close); - } - - private JobContext getOrCreateContext(Long jobId, DataSource dataSource) { + private JobContext getOrCreateContext( + Long jobId, DataSource dataSource, Map config) { Objects.requireNonNull(jobId, "jobId"); Objects.requireNonNull(dataSource, "dataSource"); - return jobContexts.computeIfAbsent(jobId, id -> new JobContext(id, dataSource)); + return jobContexts.computeIfAbsent(jobId, id -> new JobContext(id, dataSource, config)); } private static final class JobContext { private final long jobId; private volatile SourceReader reader; - private volatile Map options; + private volatile Map config; private volatile DataSource dataSource; - private JobContext(long jobId, DataSource dataSource) { + private JobContext(long jobId, DataSource dataSource, Map config) { this.jobId = jobId; this.dataSource = dataSource; + this.config = config; } private synchronized SourceReader getOrCreateReader(DataSource source) { @@ -113,10 +110,6 @@ private JobContext(long jobId, DataSource dataSource) { return reader; } - private void setOptions(Map options) { - this.options = options; - } - private void close() { if (reader != null) { reader.close(jobId); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java index 4052166b99591c..6e0f744e262ccb 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java @@ -26,5 +26,4 @@ public class LoadConstants { public static final String DATABASE_NAME = "database"; public static final String INCLUDE_TABLES_LIST = "include_tables"; public static final String EXCLUDE_TABLES_LIST = "exclude_tables"; - public static final String INCLUDE_SCHEMA_CHANGES = "include_schema_changes"; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index f24e6bc14cb2b7..4b56fda55f244a 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -17,15 +17,19 @@ package org.apache.doris.cdcclient.controller; -import org.apache.doris.cdcclient.common.JobManager; -import org.apache.doris.cdcclient.model.FetchRecordReq; +import org.apache.doris.cdcclient.common.Env; import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.cdcclient.model.req.FetchRecordReq; +import org.apache.doris.cdcclient.model.req.WriteRecordReq; +import org.apache.doris.cdcclient.model.resp.RecordWithMeta; +import org.apache.doris.cdcclient.model.resp.WriteMetaResp; import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; -import org.apache.doris.cdcclient.source.factory.DataSource; +import org.apache.doris.cdcclient.service.PipelineCoordinator; import org.apache.doris.cdcclient.source.reader.SourceReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.web.bind.annotation.PathVariable; import org.springframework.web.bind.annotation.RequestBody; import org.springframework.web.bind.annotation.RequestMapping; @@ -33,24 +37,21 @@ import org.springframework.web.bind.annotation.RestController; import java.util.List; -import java.util.Locale; -import java.util.Map; @RestController public class ClientController { private static final Logger LOG = LoggerFactory.getLogger(ClientController.class); - /** - * Fetch source splits for snapshot - * @param config - * @return - * @throws Exception - */ + @Autowired private PipelineCoordinator pipelineCoordinator; + + /** Fetch source splits for snapshot */ @RequestMapping(path = "/api/fetchSplits", method = RequestMethod.POST) public Object fetchSplits(@RequestBody JobConfig config) throws Exception { try { - SourceReader reader = - prepareReader(config.getJobId(), config.getDataSource(), config.getConfig()); + SourceReader reader = + Env.getCurrentEnv() + .getReader( + config.getJobId(), config.getDataSource(), config.getConfig()); List splits = reader.getSourceSplits(config); return ResponseEntityBuilder.ok(splits); } catch (IllegalArgumentException ex) { @@ -59,17 +60,11 @@ public Object fetchSplits(@RequestBody JobConfig config) throws Exception { } } - /** - * Fetch records from source reader - * @param recordReq - * @return - */ + /** Fetch records from source reader */ @RequestMapping(path = "/api/fetchRecords", method = RequestMethod.POST) public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { try { - SourceReader reader = - prepareReader(recordReq.getJobId(), recordReq.getDataSource(), recordReq.getConfig()); - Object response = reader.read(recordReq); + RecordWithMeta response = pipelineCoordinator.read(recordReq); return ResponseEntityBuilder.ok(response); } catch (Exception ex) { LOG.error("Failed fetch record, jobId={}", recordReq.getJobId(), ex); @@ -77,40 +72,23 @@ public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { } } - /** - * Write records to backend - * @param recordReq - * @return - */ - @RequestMapping(path = "/api/writeRecord", method = RequestMethod.POST) - public Object writeRecord(@RequestBody FetchRecordReq recordReq) { - // 拉取数据,写入到backend,这里先不实现 - return null; + /** Fetch records from source reader and Write records to backend */ + @RequestMapping(path = "/api/writeRecords", method = RequestMethod.POST) + public Object writeRecord(@RequestBody WriteRecordReq recordReq) { + try { + WriteMetaResp response = pipelineCoordinator.readAndWrite(recordReq); + return ResponseEntityBuilder.ok(response); + } catch (Exception ex) { + LOG.error("Failed to write record, jobId={}", recordReq.getJobId(), ex); + return ResponseEntityBuilder.badRequest(ex.getMessage()); + } } @RequestMapping(path = "/api/close/{jobId}", method = RequestMethod.POST) public Object close(@PathVariable long jobId) { - JobManager manager = JobManager.getInstance(); - manager.close(jobId); + Env env = Env.getCurrentEnv(); + env.close(jobId); + pipelineCoordinator.closeJob(jobId); return ResponseEntityBuilder.ok(true); } - - private DataSource resolveDataSource(String source) { - if (source == null || source.trim().isEmpty()) { - throw new IllegalArgumentException("Missing dataSource"); - } - try { - return DataSource.valueOf(source.trim().toUpperCase(Locale.ROOT)); - } catch (IllegalArgumentException ex) { - throw new IllegalArgumentException("Unsupported dataSource: " + source, ex); - } - } - - private SourceReader prepareReader( - Long jobId, String dataSource, Map config) { - DataSource parsed = resolveDataSource(dataSource); - JobManager manager = JobManager.getInstance(); - manager.attachOptions(jobId, parsed, config); - return manager.getOrCreateReader(jobId, parsed); - } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FinishSplitsReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/StreamLoadException.java similarity index 51% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FinishSplitsReq.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/StreamLoadException.java index 1d4e09ff8eb229..4c344e19799dab 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FinishSplitsReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/StreamLoadException.java @@ -15,36 +15,30 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model; +package org.apache.doris.cdcclient.exception; -import java.util.Map; - -public class FinishSplitsReq { - - private String datasource; - - private Map> finishedOffsets; - - public FinishSplitsReq() {} - - public FinishSplitsReq(String datasource, Map> finishedOffsets) { - this.datasource = datasource; - this.finishedOffsets = finishedOffsets; +public class StreamLoadException extends RuntimeException { + public StreamLoadException() { + super(); } - public String getDatasource() { - return datasource; + public StreamLoadException(String message) { + super(message); } - public void setDatasource(String datasource) { - this.datasource = datasource; + public StreamLoadException(String message, Throwable cause) { + super(message, cause); } - public Map> getFinishedOffsets() { - return finishedOffsets; + public StreamLoadException(Throwable cause) { + super(cause); } - public void setFinishedOffsets(Map> finishedOffsets) { - this.finishedOffsets = finishedOffsets; + protected StreamLoadException( + String message, + Throwable cause, + boolean enableSuppression, + boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java index 312fa582cff88a..921f26449605a6 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java @@ -18,42 +18,15 @@ package org.apache.doris.cdcclient.model; import java.util.Map; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +@Data +@AllArgsConstructor +@NoArgsConstructor public class JobConfig { - private Long jobId; private String dataSource; private Map config; - - public JobConfig() {} - - public JobConfig(Long jobId, String dataSource, Map config) { - this.jobId = jobId; - this.dataSource = dataSource; - this.config = config; - } - - public Long getJobId() { - return jobId; - } - - public void setJobId(Long jobId) { - this.jobId = jobId; - } - - public String getDataSource() { - return dataSource; - } - - public void setDataSource(String dataSource) { - this.dataSource = dataSource; - } - - public Map getConfig() { - return config; - } - - public void setConfig(Map config) { - this.config = config; - } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FetchRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/BaseRecordReq.java similarity index 51% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FetchRecordReq.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/BaseRecordReq.java index df21b8841537a0..545756ea74066d 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/FetchRecordReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/BaseRecordReq.java @@ -15,63 +15,26 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model; +package org.apache.doris.cdcclient.model.req; + +import lombok.Getter; +import lombok.Setter; import java.util.Map; -public class FetchRecordReq { - private boolean reload; - private int fetchSize; +@Getter +@Setter +public abstract class BaseRecordReq { private long jobId; private String dataSource; private Map meta; private Map config; public boolean isReload() { - return reload; - } - - public void setReload(boolean reload) { - this.reload = reload; + return false; } public int getFetchSize() { - return fetchSize; - } - - public void setFetchSize(int fetchSize) { - this.fetchSize = fetchSize; - } - - public long getJobId() { - return jobId; - } - - public void setJobId(long jobId) { - this.jobId = jobId; - } - - public String getDataSource() { - return dataSource; - } - - public void setDataSource(String dataSource) { - this.dataSource = dataSource; - } - - public Map getMeta() { - return meta; - } - - public void setMeta(Map meta) { - this.meta = meta; - } - - public Map getConfig() { - return config; - } - - public void setConfig(Map config) { - this.config = config; + return Integer.MAX_VALUE; } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/FetchRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/FetchRecordReq.java new file mode 100644 index 00000000000000..2a829fab29b1c0 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/FetchRecordReq.java @@ -0,0 +1,38 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model.req; + +import lombok.Data; +import lombok.EqualsAndHashCode; + +@Data +@EqualsAndHashCode(callSuper = true) +public class FetchRecordReq extends BaseRecordReq { + private boolean reload; + private int fetchSize; + + @Override + public boolean isReload() { + return reload; + } + + @Override + public int getFetchSize() { + return fetchSize; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/WriteRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/WriteRecordReq.java new file mode 100644 index 00000000000000..1f59cfdf9795aa --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/WriteRecordReq.java @@ -0,0 +1,25 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model.req; + +import lombok.Data; +import lombok.EqualsAndHashCode; + +@Data +@EqualsAndHashCode(callSuper = true) +public class WriteRecordReq extends BaseRecordReq {} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/RecordWithMeta.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/RecordWithMeta.java similarity index 68% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/RecordWithMeta.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/RecordWithMeta.java index 9c5dd5c7360ef8..f4bc38ea0615ef 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/RecordWithMeta.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/RecordWithMeta.java @@ -15,13 +15,15 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.source.reader; +package org.apache.doris.cdcclient.model.resp; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import lombok.Data; +@Data public class RecordWithMeta { private Map meta; private List records; @@ -30,26 +32,4 @@ public RecordWithMeta() { this.meta = new HashMap<>(); this.records = new ArrayList<>(); } - - public RecordWithMeta(Map meta, List records) { - this.meta = meta; - this.records = records; - } - - public Map getMeta() { - return meta; - } - - public void setMeta(Map meta) { - this.meta = meta; - } - - public List getRecords() { - return records; - } - - public void setRecords(List records) { - this.records = records; - } - } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/WriteMetaResp.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/WriteMetaResp.java new file mode 100644 index 00000000000000..4004611c8bde64 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/WriteMetaResp.java @@ -0,0 +1,9 @@ +package org.apache.doris.cdcclient.model.resp; + +import java.util.Map; +import lombok.Data; + +@Data +public class WriteMetaResp { + private Map meta; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java new file mode 100644 index 00000000000000..51d9f79511658f --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -0,0 +1,181 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.service; + +import org.apache.doris.cdcclient.common.Env; +import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.cdcclient.model.req.FetchRecordReq; +import org.apache.doris.cdcclient.model.req.WriteRecordReq; +import org.apache.doris.cdcclient.model.resp.RecordWithMeta; +import org.apache.doris.cdcclient.model.resp.WriteMetaResp; +import org.apache.doris.cdcclient.sink.DorisBatchStreamLoad; +import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; +import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; +import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.cdcclient.source.reader.SplitReadResult; + +import io.debezium.data.Envelope; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Component; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** Pipeline coordinator. */ +@Component +public class PipelineCoordinator { + private static final Logger LOG = LoggerFactory.getLogger(PipelineCoordinator.class); + private static final String SPLIT_ID = "splitId"; + private static final String PURE_BINLOG_PHASE = "pureBinlogPhase"; + + // jobId + private final Map batchStreamLoadMap = new ConcurrentHashMap<>(); + private final SourceRecordDeserializer> serializer; + + public PipelineCoordinator() { + this.serializer = new DebeziumJsonDeserializer(); + } + + /** Read data from SourceReader and return it with meta information. */ + public RecordWithMeta read(FetchRecordReq recordReq) throws Exception { + SourceReader reader = + Env.getCurrentEnv() + .getReader( + recordReq.getJobId(), + recordReq.getDataSource(), + recordReq.getConfig()); + return reader.read(recordReq); + } + + /** Read data from SourceReader and write it to Doris, while returning meta information. */ + public WriteMetaResp readAndWrite(WriteRecordReq writeRecordReq) throws Exception { + SourceReader sourceReader = + Env.getCurrentEnv() + .getReader( + writeRecordReq.getJobId(), + writeRecordReq.getDataSource(), + writeRecordReq.getConfig()); + JobConfig jobConfig = + new JobConfig( + writeRecordReq.getJobId(), + writeRecordReq.getDataSource(), + writeRecordReq.getConfig()); + + Map offsetMeta = writeRecordReq.getMeta(); + WriteMetaResp recordResponse = new WriteMetaResp(); + + SplitReadResult readResult = sourceReader.readSplitRecords(writeRecordReq); + + if (readResult == null || readResult.isEmpty()) { + setDefaultMeta(recordResponse, offsetMeta, readResult); + return recordResponse; + } + + DorisBatchStreamLoad batchStreamLoad = + getOrCreateBatchStreamLoad(writeRecordReq.getJobId()); + boolean readBinlog = readResult.isReadBinlog(); + boolean pureBinlogPhase = readResult.isPureBinlogPhase(); + + // 直接使用迭代器,边读边写 + Iterator iterator = readResult.getRecordIterator(); + while (iterator != null && iterator.hasNext()) { + SourceRecord element = iterator.next(); + if (RecordUtils.isDataChangeRecord(element)) { + List serializedRecords = + serializer.deserialize(jobConfig.getConfig(), element); + if (!CollectionUtils.isEmpty(serializedRecords)) { + String database = "doris_cdc"; // doris database + String table = extractTable(element); + + for (String record : serializedRecords) { + batchStreamLoad.writeRecord(database, table, record.getBytes()); + } + + Map lastMeta = + RecordUtils.getBinlogPosition(element).getOffset(); + if (readBinlog && readResult.getSplitId() != null) { + lastMeta.put(SPLIT_ID, readResult.getSplitId()); + lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + } + recordResponse.setMeta(lastMeta); + } + } + } + + // wait stream load finish + batchStreamLoad.forceFlush(); + return recordResponse; + } + + private DorisBatchStreamLoad getOrCreateBatchStreamLoad(Long jobId) { + return batchStreamLoadMap.computeIfAbsent( + jobId, + k -> { + LOG.info("Create DorisBatchStreamLoad for jobId={}", jobId); + return new DorisBatchStreamLoad(); + }); + } + + public void closeJob(Long jobId) { + DorisBatchStreamLoad batchStreamLoad = batchStreamLoadMap.remove(jobId); + if (batchStreamLoad != null) { + LOG.info("Close DorisBatchStreamLoad for jobId={}", jobId); + batchStreamLoad.close(); + } + } + + private void setDefaultMeta( + WriteMetaResp recordResponse, + Map offsetMeta, + SplitReadResult readResult) { + if (readResult == null) { + recordResponse.setMeta(offsetMeta); + return; + } + + boolean readBinlog = readResult.isReadBinlog(); + if (readBinlog) { + Map offsetRes; + if (readResult.getDefaultOffset() != null) { + offsetRes = new HashMap<>(readResult.getDefaultOffset()); + } else { + offsetRes = new HashMap<>(offsetMeta); + } + if (readResult.getSplitId() != null) { + offsetRes.put(SPLIT_ID, readResult.getSplitId()); + } + offsetRes.put(PURE_BINLOG_PHASE, String.valueOf(readResult.isPureBinlogPhase())); + recordResponse.setMeta(offsetRes); + } else { + recordResponse.setMeta(offsetMeta); + } + } + + private String extractTable(SourceRecord record) { + Struct value = (Struct) record.value(); + return value.getStruct(Envelope.FieldName.SOURCE).getString("table"); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferHttpEntity.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferHttpEntity.java new file mode 100644 index 00000000000000..aa93b15cd89f92 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferHttpEntity.java @@ -0,0 +1,75 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; +import org.apache.http.entity.AbstractHttpEntity; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BatchBufferHttpEntity extends AbstractHttpEntity { + + private static final Logger LOG = LoggerFactory.getLogger(BatchBufferHttpEntity.class); + protected static final int OUTPUT_BUFFER_SIZE = 4096; + private final List buffer; + private final long contentLength; + + public BatchBufferHttpEntity(BatchRecordBuffer recordBuffer) { + this.buffer = recordBuffer.getBuffer(); + this.contentLength = recordBuffer.getBufferSizeBytes(); + } + + @Override + public boolean isRepeatable() { + return true; + } + + @Override + public boolean isChunked() { + return false; + } + + @Override + public long getContentLength() { + return contentLength; + } + + @Override + public InputStream getContent() { + return new BatchBufferStream(buffer); + } + + @Override + public void writeTo(OutputStream outStream) throws IOException { + try (InputStream inStream = new BatchBufferStream(buffer)) { + final byte[] buffer = new byte[OUTPUT_BUFFER_SIZE]; + int readLen; + while ((readLen = inStream.read(buffer)) != -1) { + outStream.write(buffer, 0, readLen); + } + } + } + + @Override + public boolean isStreaming() { + return false; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferStream.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferStream.java new file mode 100644 index 00000000000000..bf83d44a846de3 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferStream.java @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Iterator; +import java.util.List; + +public class BatchBufferStream extends InputStream { + private final Iterator iterator; + private byte[] currentRow; + private int currentPos; + + public BatchBufferStream(List buffer) { + this.iterator = buffer.iterator(); + } + + @Override + public int read() throws IOException { + return 0; + } + + @Override + public int read(byte[] buf) throws IOException { + return read(buf, 0, buf.length); + } + + @Override + public int read(byte[] buf, int off, int len) throws IOException { + if (!iterator.hasNext() && currentRow == null) { + return -1; + } + + byte[] item = currentRow; + int pos = currentPos; + int readBytes = 0; + while (readBytes < len && (item != null || iterator.hasNext())) { + if (item == null) { + item = iterator.next(); + pos = 0; + } + + int size = Math.min(len - readBytes, item.length - pos); + System.arraycopy(item, pos, buf, off + readBytes, size); + readBytes += size; + pos += size; + + if (pos == item.length) { + item = null; + pos = 0; + } + } + currentRow = item; + currentPos = pos; + return readBytes; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java new file mode 100644 index 00000000000000..21ddf4bf82fa60 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java @@ -0,0 +1,155 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import java.util.LinkedList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** buffer to queue. */ +public class BatchRecordBuffer { + private static final Logger LOG = LoggerFactory.getLogger(BatchRecordBuffer.class); + public static final String LINE_SEPARATOR = "\n"; + private String labelName; + private LinkedList buffer; + private byte[] lineDelimiter; + private int numOfRecords = 0; + private long bufferSizeBytes = 0; + private boolean loadBatchFirstRecord = true; + private String database; + private String table; + private final long createTime = System.currentTimeMillis(); + private long retainTime = 0; + + public BatchRecordBuffer() { + this.buffer = new LinkedList<>(); + } + + public BatchRecordBuffer(String database, String table, byte[] lineDelimiter, long retainTime) { + super(); + this.database = database; + this.table = table; + this.lineDelimiter = lineDelimiter; + this.buffer = new LinkedList<>(); + this.retainTime = retainTime; + } + + public int insert(byte[] record) { + int recordSize = record.length; + if (loadBatchFirstRecord) { + loadBatchFirstRecord = false; + } else if (lineDelimiter != null) { + this.buffer.add(this.lineDelimiter); + setBufferSizeBytes(this.bufferSizeBytes + this.lineDelimiter.length); + recordSize += this.lineDelimiter.length; + } + this.buffer.add(record); + setNumOfRecords(this.numOfRecords + 1); + setBufferSizeBytes(this.bufferSizeBytes + record.length); + return recordSize; + } + + public String getLabelName() { + return labelName; + } + + public void setLabelName(String labelName) { + this.labelName = labelName; + } + + /** + * @return true if buffer is empty + */ + public boolean isEmpty() { + return numOfRecords == 0; + } + + public void clear() { + this.buffer.clear(); + this.numOfRecords = 0; + this.bufferSizeBytes = 0; + this.labelName = null; + this.loadBatchFirstRecord = true; + } + + public LinkedList getBuffer() { + return buffer; + } + + /** + * @return Number of records in this buffer + */ + public int getNumOfRecords() { + return numOfRecords; + } + + /** + * @return Buffer size in bytes + */ + public long getBufferSizeBytes() { + return bufferSizeBytes; + } + + /** + * @param numOfRecords Updates number of records (Usually by 1) + */ + public void setNumOfRecords(int numOfRecords) { + this.numOfRecords = numOfRecords; + } + + /** + * @param bufferSizeBytes Updates sum of size of records present in this buffer (Bytes) + */ + public void setBufferSizeBytes(long bufferSizeBytes) { + this.bufferSizeBytes = bufferSizeBytes; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getTable() { + return table; + } + + public void setTable(String table) { + this.table = table; + } + + public String getTableIdentifier() { + if (database != null && table != null) { + return database + "." + table; + } + return null; + } + + public byte[] getLineDelimiter() { + return lineDelimiter; + } + + public boolean shouldFlush() { + // When the buffer create time is later than the first interval trigger, + // the write will not be triggered in the next interval, + // so multiply it by 1.5 to trigger it as early as possible. + return (System.currentTimeMillis() - createTime) * 1.5 > retainTime; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java new file mode 100644 index 00000000000000..42fcef33d7958e --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java @@ -0,0 +1,482 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.lang3.StringUtils; +import org.apache.doris.cdcclient.exception.StreamLoadException; +import org.apache.doris.cdcclient.utils.HttpUtil; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.http.HttpHeaders; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpPut; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.util.EntityUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** async stream load. */ +public class DorisBatchStreamLoad implements Serializable { + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(DorisBatchStreamLoad.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final List DORIS_SUCCESS_STATUS = + new ArrayList<>(Arrays.asList("Success")); + private static final long STREAM_LOAD_MAX_BYTES = 1 * 1024 * 1024 * 1024L; // 1 GB + private static final long STREAM_LOAD_MAX_ROWS = Integer.MAX_VALUE; + private final byte[] lineDelimiter = "\n".getBytes(); + ; + private static final String LOAD_URL_PATTERN = "http://%s/api/%s/%s/_stream_load"; + private String loadUrl; + private String hostPort; + private Map bufferMap = new ConcurrentHashMap<>(); + private ExecutorService loadExecutorService; + private LoadAsyncExecutor loadAsyncExecutor; + private BlockingQueue flushQueue; + private final AtomicBoolean started; + private volatile boolean loadThreadAlive = false; + private AtomicReference exception = new AtomicReference<>(null); + private long maxBlockedBytes; + private final AtomicLong currentCacheBytes = new AtomicLong(0L); + private final Lock lock = new ReentrantLock(); + private final Condition block = lock.newCondition(); + private final Map bufferMapLock = new ConcurrentHashMap<>(); + private int FLUSH_QUEUE_SIZE = 1; + private int FLUSH_MAX_BYTE_SIZE = 100 * 1024 * 1024; + private int FLUSH_INTERVAL_MS = 10 * 1000; + private int RETRY = 3; + + public DorisBatchStreamLoad() { + this.hostPort = "10.16.10.6:28747"; + this.flushQueue = new LinkedBlockingDeque<>(1); + // maxBlockedBytes ensures that a buffer can be written even if the queue is full + this.maxBlockedBytes = (long) FLUSH_MAX_BYTE_SIZE * (FLUSH_QUEUE_SIZE + 1); + this.loadAsyncExecutor = new LoadAsyncExecutor(FLUSH_QUEUE_SIZE); + this.loadExecutorService = + new ThreadPoolExecutor( + 1, + 1, + 0L, + TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(1), + new DefaultThreadFactory("stream-load-executor"), + new ThreadPoolExecutor.AbortPolicy()); + this.started = new AtomicBoolean(true); + this.loadExecutorService.execute(loadAsyncExecutor); + } + + /** + * write record into cache. + * + * @param record + * @throws IOException + */ + public void writeRecord(String database, String table, byte[] record) { + checkFlushException(); + String bufferKey = getTableIdentifier(database, table); + + getLock(bufferKey).readLock().lock(); + BatchRecordBuffer buffer = + bufferMap.computeIfAbsent( + bufferKey, + k -> + new BatchRecordBuffer( + database, table, this.lineDelimiter, FLUSH_INTERVAL_MS)); + + int bytes = buffer.insert(record); + currentCacheBytes.addAndGet(bytes); + getLock(bufferKey).readLock().unlock(); + + if (currentCacheBytes.get() > maxBlockedBytes) { + lock.lock(); + try { + while (currentCacheBytes.get() >= maxBlockedBytes) { + checkFlushException(); + LOG.info( + "Cache full, waiting for flush, currentBytes: {}, maxBlockedBytes: {}", + currentCacheBytes.get(), + maxBlockedBytes); + block.await(1, TimeUnit.SECONDS); + } + } catch (InterruptedException e) { + this.exception.set(e); + throw new RuntimeException(e); + } finally { + lock.unlock(); + } + } + } + + public boolean forceFlush() { + return doFlush(null, true, false); + } + + private synchronized boolean doFlush( + String bufferKey, boolean waitUtilDone, boolean bufferFull) { + checkFlushException(); + if (waitUtilDone || bufferFull) { + boolean flush = flush(bufferKey, waitUtilDone); + return flush; + } else if (flushQueue.size() < FLUSH_QUEUE_SIZE) { + boolean flush = flush(bufferKey, false); + return flush; + } + return false; + } + + private synchronized boolean flush(String bufferKey, boolean waitUtilDone) { + if (!waitUtilDone && bufferMap.isEmpty()) { + // bufferMap may have been flushed by other threads + LOG.info("bufferMap is empty, no need to flush {}", bufferKey); + return false; + } + if (null == bufferKey) { + boolean flush = false; + for (String key : bufferMap.keySet()) { + BatchRecordBuffer buffer = bufferMap.get(key); + if (waitUtilDone || buffer.shouldFlush()) { + // Ensure that the interval satisfies intervalMS + flushBuffer(key); + flush = true; + } + } + if (!waitUtilDone && !flush) { + return false; + } + } else if (bufferMap.containsKey(bufferKey)) { + flushBuffer(bufferKey); + } else { + LOG.warn("buffer not found for key: {}, may be already flushed.", bufferKey); + } + if (waitUtilDone) { + waitAsyncLoadFinish(); + } + return true; + } + + private synchronized void flushBuffer(String bufferKey) { + BatchRecordBuffer buffer; + try { + getLock(bufferKey).writeLock().lock(); + buffer = bufferMap.remove(bufferKey); + } finally { + getLock(bufferKey).writeLock().unlock(); + } + if (buffer == null) { + LOG.info("buffer key is not exist {}, skipped", bufferKey); + return; + } + buffer.setLabelName(UUID.randomUUID().toString()); + LOG.debug("flush buffer for key {} with label {}", bufferKey, buffer.getLabelName()); + putRecordToFlushQueue(buffer); + } + + private void putRecordToFlushQueue(BatchRecordBuffer buffer) { + checkFlushException(); + if (!loadThreadAlive) { + throw new RuntimeException("load thread already exit, write was interrupted"); + } + try { + flushQueue.put(buffer); + } catch (InterruptedException e) { + throw new RuntimeException("Failed to put record buffer to flush queue"); + } + // When the load thread reports an error, the flushQueue will be cleared, + // and need to force a check for the exception. + checkFlushException(); + } + + private void checkFlushException() { + if (exception.get() != null) { + throw new StreamLoadException(exception.get()); + } + } + + private void waitAsyncLoadFinish() { + // Because the queue will have a drainTo operation, it needs to be multiplied by 2 + for (int i = 0; i < FLUSH_QUEUE_SIZE * 2 + 1; i++) { + // eof buffer + BatchRecordBuffer empty = new BatchRecordBuffer(); + putRecordToFlushQueue(empty); + } + } + + private String getTableIdentifier(String database, String table) { + return database + "." + table; + } + + public void close() { + // close async executor + this.loadExecutorService.shutdown(); + this.started.set(false); + } + + @VisibleForTesting + public boolean mergeBuffer(List recordList, BatchRecordBuffer buffer) { + boolean merge = false; + if (recordList.size() > 1) { + boolean sameTable = + recordList.stream() + .map(BatchRecordBuffer::getTableIdentifier) + .distinct() + .count() + == 1; + // Buffers can be merged only if they belong to the same table. + if (sameTable) { + for (BatchRecordBuffer recordBuffer : recordList) { + if (recordBuffer != null + && recordBuffer.getLabelName() != null + && !buffer.getLabelName().equals(recordBuffer.getLabelName()) + && !recordBuffer.getBuffer().isEmpty()) { + merge(buffer, recordBuffer); + merge = true; + } + } + LOG.info( + "merge {} buffer to one stream load, result bufferBytes {}", + recordList.size(), + buffer.getBufferSizeBytes()); + } + } + return merge; + } + + private boolean merge(BatchRecordBuffer mergeBuffer, BatchRecordBuffer buffer) { + if (buffer.getBuffer().isEmpty()) { + return false; + } + if (!mergeBuffer.getBuffer().isEmpty()) { + mergeBuffer.getBuffer().add(mergeBuffer.getLineDelimiter()); + mergeBuffer.setBufferSizeBytes( + mergeBuffer.getBufferSizeBytes() + mergeBuffer.getLineDelimiter().length); + currentCacheBytes.addAndGet(buffer.getLineDelimiter().length); + } + mergeBuffer.getBuffer().addAll(buffer.getBuffer()); + mergeBuffer.setNumOfRecords(mergeBuffer.getNumOfRecords() + buffer.getNumOfRecords()); + mergeBuffer.setBufferSizeBytes( + mergeBuffer.getBufferSizeBytes() + buffer.getBufferSizeBytes()); + return true; + } + + private ReadWriteLock getLock(String bufferKey) { + return bufferMapLock.computeIfAbsent(bufferKey, k -> new ReentrantReadWriteLock()); + } + + class LoadAsyncExecutor implements Runnable { + + private int flushQueueSize; + + public LoadAsyncExecutor(int flushQueueSize) { + this.flushQueueSize = flushQueueSize; + } + + @Override + public void run() { + LOG.info("LoadAsyncExecutor start"); + loadThreadAlive = true; + List recordList = new ArrayList<>(flushQueueSize); + while (started.get()) { + recordList.clear(); + try { + BatchRecordBuffer buffer = flushQueue.poll(2000L, TimeUnit.MILLISECONDS); + if (buffer == null) { + continue; + } + if (buffer.getLabelName() == null) { + // When the label is empty, it is the eof buffer for checkpoint flush. + continue; + } + + recordList.add(buffer); + boolean merge = false; + if (!flushQueue.isEmpty()) { + flushQueue.drainTo(recordList, flushQueueSize - 1); + if (mergeBuffer(recordList, buffer)) { + load(buffer.getLabelName(), buffer); + merge = true; + } + } + + if (!merge) { + for (BatchRecordBuffer bf : recordList) { + if (bf == null || bf.getLabelName() == null) { + // When the label is empty, it's eof buffer for checkpointFlush. + continue; + } + load(bf.getLabelName(), bf); + } + } + } catch (Exception e) { + LOG.error("worker running error", e); + exception.set(e); + // clear queue to avoid writer thread blocking + flushQueue.clear(); + break; + } + } + LOG.info("LoadAsyncExecutor stop"); + loadThreadAlive = false; + } + + /** execute stream load. */ + public void load(String label, BatchRecordBuffer buffer) throws IOException { + BatchBufferHttpEntity entity = new BatchBufferHttpEntity(buffer); + HttpPut put = + new HttpPut( + String.format( + LOAD_URL_PATTERN, + hostPort, + buffer.getDatabase(), + buffer.getTable())); + put.addHeader(HttpHeaders.EXPECT, "100-continue"); + put.addHeader("read_json_by_line", "true"); + put.addHeader("format", "json"); + put.addHeader( + HttpHeaders.AUTHORIZATION, + "Basic " + + new String( + Base64.encodeBase64("root:".getBytes(StandardCharsets.UTF_8)))); + put.setEntity(entity); + + Throwable resEx = new Throwable(); + int retry = 0; + while (retry <= RETRY) { + try (CloseableHttpClient httpClient = HttpUtil.getHttpClient()) { + try (CloseableHttpResponse response = httpClient.execute(put)) { + int statusCode = response.getStatusLine().getStatusCode(); + String reason = response.getStatusLine().toString(); + if (statusCode == 200 && response.getEntity() != null) { + String loadResult = EntityUtils.toString(response.getEntity()); + LOG.info("load Result {}", loadResult); + RespContent respContent = + OBJECT_MAPPER.readValue(loadResult, RespContent.class); + if (DORIS_SUCCESS_STATUS.contains(respContent.getStatus())) { + long cacheByteBeforeFlush = + currentCacheBytes.getAndAdd(-respContent.getLoadBytes()); + LOG.info( + "load success, cacheBeforeFlushBytes: {}, currentCacheBytes : {}", + cacheByteBeforeFlush, + currentCacheBytes.get()); + lock.lock(); + try { + block.signal(); + } finally { + lock.unlock(); + } + return; + } else { + String errMsg = null; + if (StringUtils.isBlank(respContent.getMessage()) + && StringUtils.isBlank(respContent.getErrorURL())) { + // sometimes stream load will not return message + errMsg = + String.format( + "stream load error, response is %s", + loadResult); + throw new StreamLoadException(errMsg); + } else { + errMsg = + String.format( + "stream load error: %s, see more in %s", + respContent.getMessage(), + respContent.getErrorURL()); + } + throw new StreamLoadException(errMsg); + } + } + LOG.error( + "stream load failed with {}, reason {}, to retry", + hostPort, + reason); + if (retry == RETRY) { + resEx = new StreamLoadException("stream load failed with: " + reason); + } + } catch (Exception ex) { + resEx = ex; + LOG.error("stream load error with {}, to retry, cause by", hostPort, ex); + } + } + retry++; + try { + Thread.sleep(retry * 1000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + buffer.clear(); + buffer = null; + + if (retry >= RETRY) { + throw new StreamLoadException("stream load error: " + resEx.getMessage(), resEx); + } + } + } + + static class DefaultThreadFactory implements ThreadFactory { + private static final AtomicInteger poolNumber = new AtomicInteger(1); + private final AtomicInteger threadNumber = new AtomicInteger(1); + private final String namePrefix; + + DefaultThreadFactory(String name) { + namePrefix = "pool-" + poolNumber.getAndIncrement() + "-" + name + "-"; + } + + public Thread newThread(Runnable r) { + Thread t = new Thread(r, namePrefix + threadNumber.getAndIncrement()); + t.setDaemon(false); + return t; + } + } + + @VisibleForTesting + public AtomicReference getException() { + return exception; + } + + @VisibleForTesting + public boolean isLoadThreadAlive() { + return loadThreadAlive; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/RespContent.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/RespContent.java new file mode 100644 index 00000000000000..0a3c8272681c93 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/RespContent.java @@ -0,0 +1,169 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.annotation.VisibleForTesting; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class RespContent { + + @JsonProperty(value = "TxnId") + private Long txnId; + + @JsonProperty(value = "Label") + private String label; + + @JsonProperty(value = "Status") + private String status; + + @JsonProperty(value = "TwoPhaseCommit") + private String twoPhaseCommit; + + @JsonProperty(value = "ExistingJobStatus") + private String existingJobStatus; + + @JsonProperty(value = "Message") + private String message; + + @JsonProperty(value = "NumberTotalRows") + private Long numberTotalRows; + + @JsonProperty(value = "NumberLoadedRows") + private Long numberLoadedRows; + + @JsonProperty(value = "NumberFilteredRows") + private Integer numberFilteredRows; + + @JsonProperty(value = "NumberUnselectedRows") + private Integer numberUnselectedRows; + + @JsonProperty(value = "LoadBytes") + private Long loadBytes; + + @JsonProperty(value = "LoadTimeMs") + private Integer loadTimeMs; + + @JsonProperty(value = "BeginTxnTimeMs") + private Integer beginTxnTimeMs; + + @JsonProperty(value = "StreamLoadPutTimeMs") + private Integer streamLoadPutTimeMs; + + @JsonProperty(value = "ReadDataTimeMs") + private Integer readDataTimeMs; + + @JsonProperty(value = "WriteDataTimeMs") + private Integer writeDataTimeMs; + + @JsonProperty(value = "CommitAndPublishTimeMs") + private Integer commitAndPublishTimeMs; + + @JsonProperty(value = "ErrorURL") + private String errorURL; + + public Long getTxnId() { + return txnId; + } + + public String getStatus() { + return status; + } + + public String getTwoPhaseCommit() { + return twoPhaseCommit; + } + + public String getMessage() { + return message; + } + + public String getExistingJobStatus() { + return existingJobStatus; + } + + public Long getNumberTotalRows() { + return numberTotalRows; + } + + public Long getNumberLoadedRows() { + return numberLoadedRows; + } + + public Integer getNumberFilteredRows() { + return numberFilteredRows; + } + + public Integer getNumberUnselectedRows() { + return numberUnselectedRows; + } + + public Long getLoadBytes() { + return loadBytes; + } + + public Integer getLoadTimeMs() { + return loadTimeMs; + } + + public Integer getBeginTxnTimeMs() { + return beginTxnTimeMs; + } + + public Integer getStreamLoadPutTimeMs() { + return streamLoadPutTimeMs; + } + + public Integer getReadDataTimeMs() { + return readDataTimeMs; + } + + public Integer getWriteDataTimeMs() { + return writeDataTimeMs; + } + + public Integer getCommitAndPublishTimeMs() { + return commitAndPublishTimeMs; + } + + public String getLabel() { + return label; + } + + @VisibleForTesting + public void setMessage(String message) { + this.message = message; + } + + @Override + public String toString() { + ObjectMapper mapper = new ObjectMapper(); + try { + return mapper.writeValueAsString(this); + } catch (JsonProcessingException e) { + return ""; + } + } + + public String getErrorURL() { + return errorURL; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/JsonSerializer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java similarity index 93% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/JsonSerializer.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java index 35d83267c6cbb6..76edb7998d0189 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/JsonSerializer.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java @@ -15,14 +15,13 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.source.serialize; +package org.apache.doris.cdcclient.source.deserialize; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import io.debezium.data.Envelope; import io.debezium.data.SpecialValueDecimal; import io.debezium.data.VariableScaleDecimal; -import io.debezium.relational.Tables; import io.debezium.time.Date; import io.debezium.time.MicroTimestamp; import io.debezium.time.NanoTimestamp; @@ -50,18 +49,17 @@ import java.util.List; import java.util.Map; -public class JsonSerializer implements DorisRecordSerializer> { +public class DebeziumJsonDeserializer + implements SourceRecordDeserializer> { private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(JsonSerializer.class); + private static final Logger LOG = LoggerFactory.getLogger(DebeziumJsonDeserializer.class); private static final String DELETE_SIGN_KEY = "__DORIS_DELETE_SIGN__"; private static ObjectMapper objectMapper = new ObjectMapper(); - private Map config; - private transient Tables tables; - public JsonSerializer() {} + public DebeziumJsonDeserializer() {} @Override - public List serialize(Map context, SourceRecord record) + public List deserialize(Map context, SourceRecord record) throws IOException { if (RecordUtils.isDataChangeRecord(record)) { LOG.trace("Process data change record: {}", record); @@ -79,19 +77,18 @@ private List deserializeDataChangeRecord(SourceRecord record) throws IOE List rows = new ArrayList<>(); Envelope.Operation op = Envelope.operationFor(record); Struct value = (Struct) record.value(); - String table = value.getStruct(Envelope.FieldName.SOURCE).getString("table"); Schema valueSchema = record.valueSchema(); if (Envelope.Operation.DELETE.equals(op)) { String deleteRow = extractBeforeRow(value, valueSchema); if (StringUtils.isNotEmpty(deleteRow)) { - rows.add(table + "|" + deleteRow); + rows.add(deleteRow); } } else if (Envelope.Operation.READ.equals(op) || Envelope.Operation.CREATE.equals(op) || Envelope.Operation.UPDATE.equals(op)) { String insertRow = extractAfterRow(value, valueSchema); if (StringUtils.isNotEmpty(insertRow)) { - rows.add(table + "|" + insertRow); + rows.add(insertRow); } } return rows; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecordSerializer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/SourceRecordDeserializer.java similarity index 81% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecordSerializer.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/SourceRecordDeserializer.java index e501c4030b8f6c..3dd72a23c65188 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecordSerializer.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/SourceRecordDeserializer.java @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.source.serialize; +package org.apache.doris.cdcclient.source.deserialize; import java.io.IOException; import java.io.Serializable; import java.util.Map; -public interface DorisRecordSerializer extends Serializable { +public interface SourceRecordDeserializer extends Serializable { - C serialize(Map context, T record) throws IOException; + C deserialize(Map context, T record) throws IOException; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java index 5c26e9a5cd9b2a..f59d11510aaeb3 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java @@ -1,21 +1,15 @@ package org.apache.doris.cdcclient.source.factory; -import org.apache.doris.cdcclient.source.reader.SourceReader; -import org.apache.doris.cdcclient.source.reader.mysql.MySqlSourceReader; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.function.Supplier; +import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.cdcclient.source.reader.mysql.MySqlSourceReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -/** - * SourceReader 注册中心,支持在启动阶段按需扩展数据源。 - */ +/** SourceReader register. */ public final class SourceReaderFactory { private static final Logger LOG = LoggerFactory.getLogger(SourceReaderFactory.class); private static final Map>> REGISTRY = @@ -34,22 +28,6 @@ public static void register(DataSource source, Supplier> supp LOG.info("Registered SourceReader provider for {}", source); } - public static void unregister(DataSource source) { - if (source == null) { - return; - } - REGISTRY.remove(source); - LOG.info("Unregistered SourceReader provider for {}", source); - } - - public static boolean supports(DataSource source) { - return source != null && REGISTRY.containsKey(source); - } - - public static Set supportedDataSources() { - return Collections.unmodifiableSet(REGISTRY.keySet()); - } - public static SourceReader createSourceReader(DataSource source) { Supplier> supplier = REGISTRY.get(source); if (supplier == null) { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index acb7a9497f66cb..d9450094a527ce 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -17,15 +17,21 @@ package org.apache.doris.cdcclient.source.reader; -import org.apache.doris.cdcclient.model.FetchRecordReq; -import org.apache.doris.cdcclient.model.JobConfig; -import org.apache.doris.cdcclient.source.split.SourceSplit; - import com.fasterxml.jackson.core.JsonProcessingException; - import java.util.List; +import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.cdcclient.model.req.BaseRecordReq; +import org.apache.doris.cdcclient.model.req.FetchRecordReq; +import org.apache.doris.cdcclient.model.resp.RecordWithMeta; +import org.apache.doris.cdcclient.source.split.SourceSplit; -public interface SourceReader { +/** + * SourceReader 接口,支持泛型以指定 Split 和 SplitState 类型。 + * + * @param Split 类型(如 MySqlSplit) + * @param SplitState 类型(如 MySqlSplitState) + */ +public interface SourceReader { /** Initialization, called when the program starts */ void initialize(); @@ -46,6 +52,18 @@ public interface SourceReader { */ RecordWithMeta read(FetchRecordReq meta) throws Exception; + /** + * Reading Data for split reader + * + * @param baseReq 基础请求 + * @return 读取结果,包含 SourceRecord 列表和状态信息 + */ + default SplitReadResult readSplitRecords(BaseRecordReq baseReq) + throws Exception { + throw new UnsupportedOperationException( + "readSplitRecords is not supported by " + this.getClass().getName()); + } + /** Called when closing */ void close(Long jobId); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java new file mode 100644 index 00000000000000..f1c06ead7c637f --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java @@ -0,0 +1,46 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.reader; + +import java.util.Iterator; +import java.util.Map; +import lombok.Data; +import org.apache.kafka.connect.source.SourceRecord; + +/** + * 保存从 split 中读取的结果。 使用泛型支持不同类型的 Split 和 SplitState。 使用迭代器模式,避免将所有数据加载到内存。 + * + * @param Split 类型(如 MySqlSplit) + * @param SplitState 类型(如 MySqlSplitState) + */ +@Data +public class SplitReadResult { + private Iterator recordIterator; // 使用迭代器,支持流式处理 + private Map lastMeta; + private SplitState splitState; // Split state,由具体的 reader 管理 + private boolean readBinlog; // 是否读取 binlog(对于支持 binlog 的数据源) + private boolean pureBinlogPhase; // 是否处于纯 binlog 阶段 + private Split split; // Split,由具体的 reader 管理 + private String splitId; // split ID + private Map defaultOffset; // 默认的 offset(用于没有数据时) + + /** 检查是否有数据(延迟检查,因为迭代器可能还没有被消费) */ + public boolean isEmpty() { + return recordIterator == null || !recordIterator.hasNext(); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java index d7c712e463d116..905aaf00bc8b33 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java @@ -17,11 +17,10 @@ package org.apache.doris.cdcclient.source.reader; +import java.util.Iterator; import org.apache.flink.cdc.connectors.mysql.source.split.SourceRecords; import org.apache.kafka.connect.source.SourceRecord; -import java.util.Iterator; - public class SplitRecords { private final String splitId; private final SourceRecords records; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index d864fe13243b74..ac8b77abaf5ad0 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -18,16 +18,19 @@ package org.apache.doris.cdcclient.source.reader.mysql; import org.apache.doris.cdcclient.constants.LoadConstants; -import org.apache.doris.cdcclient.model.FetchRecordReq; import org.apache.doris.cdcclient.model.JobConfig; -import org.apache.doris.cdcclient.mysql.utils.ConfigUtil; +import org.apache.doris.cdcclient.model.req.BaseRecordReq; +import org.apache.doris.cdcclient.model.req.FetchRecordReq; +import org.apache.doris.cdcclient.model.resp.RecordWithMeta; +import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; +import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; +import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.cdcclient.source.reader.SplitReadResult; import org.apache.doris.cdcclient.source.reader.SplitRecords; -import org.apache.doris.cdcclient.source.reader.RecordWithMeta; -import org.apache.doris.cdcclient.source.serialize.DorisRecordSerializer; -import org.apache.doris.cdcclient.source.serialize.JsonSerializer; import org.apache.doris.cdcclient.source.split.AbstractSourceSplit; import org.apache.doris.cdcclient.source.split.BinlogSplit; import org.apache.doris.cdcclient.source.split.SnapshotSplit; +import org.apache.doris.cdcclient.utils.ConfigUtil; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; @@ -77,12 +80,13 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.Objects; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; -public class MySqlSourceReader implements org.apache.doris.cdcclient.source.reader.SourceReader { +public class MySqlSourceReader implements SourceReader { private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceReader.class); private static ObjectMapper objectMapper = new ObjectMapper(); private static final String SPLIT_ID = "splitId"; @@ -92,11 +96,11 @@ public class MySqlSourceReader implements org.apache.doris.cdcclient.source.read private static final String PURE_BINLOG_PHASE = "pureBinlogPhase"; private static final FlinkJsonTableChangeSerializer TABLE_CHANGE_SERIALIZER = new FlinkJsonTableChangeSerializer(); - private DorisRecordSerializer> serializer; + private SourceRecordDeserializer> serializer; private Map jobContexts; public MySqlSourceReader() { - this.serializer = new JsonSerializer(); + this.serializer = new DebeziumJsonDeserializer(); this.jobContexts = new ConcurrentHashMap<>(); } @@ -152,49 +156,35 @@ public List getSourceSplits(JobConfig config) } /** - * 1. If the SplitRecords iterator has it, read the iterator directly. 2. If there is a - * binlogreader, poll it. 3. If there is none, resubmit split. 4. If reload is true, need to - * reset binlogSplitReader and submit split. - * - * @param fetchRecord - * @return - * @throws Exception + * 1. If the SplitRecords iterator has it, read the iterator directly. + * 2. If there is a binlogreader, poll it. + * 3. If there is none, resubmit split. 4. If reload is true, need to + * reset binlogSplitReader and submit split. */ @Override public RecordWithMeta read(FetchRecordReq fetchRecord) throws Exception { + SplitReadResult readResult = readSplitRecords(fetchRecord); + JobConfig jobConfig = - new JobConfig(fetchRecord.getJobId(), fetchRecord.getDataSource(), fetchRecord.getConfig()); - LOG.debug( - "Start read for jobId={}, dataSource={}, reload={}", - jobConfig.getJobId(), - jobConfig.getDataSource(), - fetchRecord.isReload()); - RecordWithMeta recordResponse = new RecordWithMeta(); + new JobConfig( + fetchRecord.getJobId(), + fetchRecord.getDataSource(), + fetchRecord.getConfig()); Map offsetMeta = fetchRecord.getMeta(); - if (offsetMeta.isEmpty()) { - throw new RuntimeException("miss meta offset"); - } - - JobRuntimeContext jobRuntimeContext = getJobRuntimeContext(jobConfig.getJobId()); - SplitProcessingContext processingContext = - prepareSplitProcessingContext(jobRuntimeContext, jobConfig, offsetMeta); - - if (fetchRecord.isReload() && processingContext.getSplit() == null) { - processingContext = - reloadSplitProcessingContext(jobRuntimeContext, jobConfig, offsetMeta); - } + RecordWithMeta recordResponse = new RecordWithMeta(); - return drainSplitRecords( - fetchRecord, jobConfig, offsetMeta, jobRuntimeContext, processingContext, recordResponse); + return buildRecordResponse(fetchRecord, jobConfig, offsetMeta, recordResponse, readResult); } /** - * 根据当前上下文准备 Split 信息: - * - 若已有正在消费的 split,直接复用; - * - 否则根据 offset 构造新的 snapshot/binlog split 并启动 reader。 + * Prepare split information based on current context: + * - If there is an active split being consumed, reuse it directly; + * - Otherwise, create a new snapshot/binlog split based on offset and start the reader. */ private SplitProcessingContext prepareSplitProcessingContext( - JobRuntimeContext jobRuntimeContext, JobConfig jobConfig, Map offsetMeta) + JobRuntimeContext jobRuntimeContext, + JobConfig jobConfig, + Map offsetMeta) throws Exception { SplitProcessingContext context = new SplitProcessingContext(); SplitRecords currentSplitRecords = jobRuntimeContext.getCurrentSplitRecords(); @@ -227,10 +217,12 @@ private SplitProcessingContext prepareSplitProcessingContext( } /** - * 当客户端请求 reload 时,重新构造 split 并重置 binlog reader。 + * When a client requests a reload, the split is reconstructed and the binlog reader is reset. */ private SplitProcessingContext reloadSplitProcessingContext( - JobRuntimeContext jobRuntimeContext, JobConfig jobConfig, Map offsetMeta) + JobRuntimeContext jobRuntimeContext, + JobConfig jobConfig, + Map offsetMeta) throws Exception { SplitProcessingContext context = new SplitProcessingContext(); Tuple2 splitFlag = createMySqlSplit(offsetMeta, jobConfig); @@ -249,56 +241,106 @@ private SplitProcessingContext reloadSplitProcessingContext( return context; } - /** - * 消费当前 split 中的记录并封装响应,同时更新元信息。 - */ - private RecordWithMeta drainSplitRecords( - FetchRecordReq fetchRecord, - JobConfig jobConfig, - Map offsetMeta, - JobRuntimeContext jobRuntimeContext, - SplitProcessingContext processingContext, - RecordWithMeta recordResponse) + /** read split records. */ + @Override + public SplitReadResult readSplitRecords(BaseRecordReq baseReq) throws Exception { - int count = 0; - int fetchSize = fetchRecord.getFetchSize(); + JobConfig jobConfig = + new JobConfig(baseReq.getJobId(), baseReq.getDataSource(), baseReq.getConfig()); + Map offsetMeta = baseReq.getMeta(); + if (offsetMeta == null || offsetMeta.isEmpty()) { + throw new RuntimeException("miss meta offset"); + } + + JobRuntimeContext jobRuntimeContext = getJobRuntimeContext(jobConfig.getJobId()); + SplitProcessingContext processingContext = + prepareSplitProcessingContext(jobRuntimeContext, jobConfig, offsetMeta); + + if (baseReq.isReload() && processingContext.getSplit() == null) { + processingContext = + reloadSplitProcessingContext(jobRuntimeContext, jobConfig, offsetMeta); + } + + return readSplitRecords(baseReq, jobRuntimeContext, processingContext); + } + + private SplitReadResult readSplitRecords( + BaseRecordReq baseReq, + JobRuntimeContext jobRuntimeContext, + SplitProcessingContext processingContext) { + int fetchSize = baseReq.getFetchSize(); SplitRecords currentSplitRecords = processingContext.getSplitRecords(); boolean readBinlog = processingContext.isReadBinlog(); boolean pureBinlogPhase = processingContext.isPureBinlogPhase(); MySqlSplit split = processingContext.getSplit(); + SplitReadResult result = new SplitReadResult<>(); MySqlSplitState currentSplitState = null; if (!readBinlog && split != null) { currentSplitState = new MySqlSnapshotSplitState(split.asSnapshotSplit()); } - if (currentSplitRecords != null && !currentSplitRecords.isEmpty()) { - Iterator recordIt = currentSplitRecords.getIterator(); - while (recordIt.hasNext()) { - SourceRecord element = recordIt.next(); - if (RecordUtils.isWatermarkEvent(element)) { - BinlogOffset watermark = RecordUtils.getWatermark(element); - if (RecordUtils.isHighWatermarkEvent(element) && currentSplitState != null) { - currentSplitState.asSnapshotSplitState().setHighWatermark(watermark); - } - } else if (RecordUtils.isHeartbeatEvent(element)) { - LOG.debug("Receive heartbeat event: {}", element); - } else if (RecordUtils.isDataChangeRecord(element)) { - count += - handleDataChangeRecord( - jobConfig, - recordResponse, - element, - readBinlog, - pureBinlogPhase); - if (count >= fetchSize) { - return recordResponse; - } - } else { - LOG.debug("Ignore event: {}", element); + + Iterator filteredIterator = + new FilteredRecordIterator( + currentSplitRecords, + currentSplitState, + fetchSize, + readBinlog, + pureBinlogPhase, + result); + + result.setRecordIterator(filteredIterator); + jobRuntimeContext.setCurrentSplitRecords(null); + result.setSplitState(currentSplitState); + result.setReadBinlog(readBinlog); + result.setPureBinlogPhase(pureBinlogPhase); + result.setSplit(split); + + // set splitId and defaultOffset + if (split != null) { + result.setSplitId(split.splitId()); + if (readBinlog && split instanceof MySqlBinlogSplit) { + result.setDefaultOffset(split.asBinlogSplit().getStartingOffset().getOffset()); + } + } else if (readBinlog) { + result.setSplitId(BINLOG_SPLIT_ID); + } + + return result; + } + + /** build RecordWithMeta */ + private RecordWithMeta buildRecordResponse( + FetchRecordReq fetchRecord, + JobConfig jobConfig, + Map offsetMeta, + RecordWithMeta recordResponse, + SplitReadResult readResult) + throws Exception { + boolean readBinlog = readResult.isReadBinlog(); + boolean pureBinlogPhase = readResult.isPureBinlogPhase(); + MySqlSplit split = readResult.getSplit(); + MySqlSplitState currentSplitState = readResult.getSplitState(); + + // Serialize records and add them to the response (collect from iterator) + Iterator iterator = readResult.getRecordIterator(); + while (iterator != null && iterator.hasNext()) { + SourceRecord element = iterator.next(); + List serializedRecords = serializer.deserialize(jobConfig.getConfig(), element); + if (!CollectionUtils.isEmpty(serializedRecords)) { + recordResponse.getRecords().addAll(serializedRecords); + + // update meta + Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); + if (readBinlog) { + lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); + lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + recordResponse.setMeta(lastMeta); } } } - jobRuntimeContext.setCurrentSplitRecords(null); + + // Set meta information if (!readBinlog && currentSplitState != null) { BinlogOffset highWatermark = currentSplitState.asSnapshotSplitState().getHighWatermark(); @@ -318,33 +360,12 @@ private RecordWithMeta drainSplitRecords( } else { recordResponse.setMeta(fetchRecord.getMeta()); } + } else if (readResult.getLastMeta() != null) { + // If there is meta from the last record, use it + recordResponse.setMeta(readResult.getLastMeta()); } - return recordResponse; - } - private int handleDataChangeRecord( - JobConfig jobConfig, - RecordWithMeta recordResponse, - SourceRecord element, - boolean readBinlog, - boolean pureBinlogPhase) - throws Exception { - List serialize = serializer.serialize(jobConfig.getConfig(), element); - if (CollectionUtils.isEmpty(serialize)) { - return 0; - } - LOG.debug( - "Job {} serialized {} records for split event", - jobConfig.getJobId(), - serialize.size()); - Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); - if (readBinlog) { - lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); - lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); - recordResponse.setMeta(lastMeta); - } - recordResponse.getRecords().addAll(serialize); - return serialize.size(); + return recordResponse; } /** @@ -674,9 +695,7 @@ private JobRuntimeContext getJobRuntimeContext(Long jobId) { return jobContexts.computeIfAbsent(jobId, JobRuntimeContext::new); } - /** - * 临时保存一次 read 调用中与 split 相关的状态,避免大量方法参数。 - */ + /** 临时保存一次 read 调用中与 split 相关的状态,避免大量方法参数。 */ private static final class SplitProcessingContext { private SplitRecords splitRecords; private MySqlSplit split; @@ -782,4 +801,89 @@ private void close() { tableSchemas = null; } } + + /** + * Filtered record iterator that only returns data change records, filtering out watermark, heartbeat and other events. + * This is a private static inner class that encapsulates record filtering logic, making the main method cleaner. + */ + private static class FilteredRecordIterator implements Iterator { + private final Iterator sourceIterator; + private final MySqlSplitState splitState; + private final int fetchSize; + private final boolean readBinlog; + private final boolean pureBinlogPhase; + private final SplitReadResult result; + + private SourceRecord nextRecord; + private int count = 0; + + FilteredRecordIterator( + SplitRecords currentSplitRecords, + MySqlSplitState splitState, + int fetchSize, + boolean readBinlog, + boolean pureBinlogPhase, + SplitReadResult result) { + this.sourceIterator = + currentSplitRecords != null && !currentSplitRecords.isEmpty() + ? currentSplitRecords.getIterator() + : null; + this.splitState = splitState; + this.fetchSize = fetchSize; + this.readBinlog = readBinlog; + this.pureBinlogPhase = pureBinlogPhase; + this.result = result; + } + + @Override + public boolean hasNext() { + if (sourceIterator == null) { + return false; + } + if (nextRecord != null) { + return true; + } + if (count >= fetchSize) { + return false; + } + + while (sourceIterator.hasNext()) { + SourceRecord element = sourceIterator.next(); + if (RecordUtils.isWatermarkEvent(element)) { + BinlogOffset watermark = RecordUtils.getWatermark(element); + if (RecordUtils.isHighWatermarkEvent(element) && splitState != null) { + splitState.asSnapshotSplitState().setHighWatermark(watermark); + } + } else if (RecordUtils.isHeartbeatEvent(element)) { + LOG.debug("Receive heartbeat event: {}", element); + } else if (RecordUtils.isDataChangeRecord(element)) { + nextRecord = element; + count++; + + // update meta + Map lastMeta = + RecordUtils.getBinlogPosition(element).getOffset(); + if (readBinlog) { + lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); + lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + result.setLastMeta(lastMeta); + } + return true; + } else { + LOG.debug("Ignore event: {}", element); + } + } + return false; + } + + @Override + public SourceRecord next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + SourceRecord record = nextRecord; + nextRecord = null; + return record; + } + } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecord.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecord.java deleted file mode 100644 index 25c4c343f24009..00000000000000 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/serialize/DorisRecord.java +++ /dev/null @@ -1,69 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.cdcclient.source.serialize; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - -public class DorisRecord implements Serializable { - public static DorisRecord EOF = new DorisRecord(null, null, Arrays.asList("EOF")); - private String database; - private String table; - private List rows; - - public DorisRecord() {} - - public DorisRecord(String database, String table, List rows) { - this.database = database; - this.table = table; - this.rows = rows; - } - - public String getDatabase() { - return database; - } - - public void setDatabase(String database) { - this.database = database; - } - - public String getTable() { - return table; - } - - public void setTable(String table) { - this.table = table; - } - - public List getRows() { - return rows; - } - - public void setRows(List rows) { - this.rows = rows; - } - - public void addRow(String row) { - this.rows.add(row); - } - - public String identifier() { - return database + "." + table; - } -} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/mysql/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java similarity index 96% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/mysql/utils/ConfigUtil.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index 6c49e94aa55a64..97555ac5844912 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/mysql/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.mysql.utils; +package org.apache.doris.cdcclient.utils; import org.apache.doris.cdcclient.constants.LoadConstants; import org.apache.doris.cdcclient.model.JobConfig; @@ -45,8 +45,7 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { configFactory.databaseList(databaseName); configFactory.serverId(String.valueOf(Math.abs(config.getJobId().hashCode()))); - configFactory.includeSchemaChanges( - Boolean.parseBoolean(cdcConfig.get(LoadConstants.INCLUDE_SCHEMA_CHANGES))); + configFactory.includeSchemaChanges(false); String includingTables = cdcConfig.getOrDefault(LoadConstants.INCLUDE_TABLES_LIST, ".*"); String includingPattern = String.format("(%s)\\.(%s)", databaseName, includingTables); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/HttpUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/HttpUtil.java new file mode 100644 index 00000000000000..7ac3d5dc2cf40a --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/HttpUtil.java @@ -0,0 +1,38 @@ +package org.apache.doris.cdcclient.utils; + +import org.apache.http.client.config.RequestConfig; +import org.apache.http.impl.NoConnectionReuseStrategy; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.DefaultRedirectStrategy; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.protocol.HttpRequestExecutor; +import org.apache.http.protocol.RequestContent; + +public class HttpUtil { + private static int connectTimeout = 30 * 1000; + private static int waitForContinueTimeout = 60 * 1000; + private static int socketTimeout = 10 * 60 * 1000; // stream load timeout 10 min + + public static CloseableHttpClient getHttpClient() { + return HttpClients.custom() + // default timeout 3s, maybe report 307 error when fe busy + .setRequestExecutor(new HttpRequestExecutor(waitForContinueTimeout)) + .setRedirectStrategy( + new DefaultRedirectStrategy() { + @Override + protected boolean isRedirectable(String method) { + return true; + } + }) + .setRetryHandler((exception, executionCount, context) -> false) + .setConnectionReuseStrategy(NoConnectionReuseStrategy.INSTANCE) + .setDefaultRequestConfig( + RequestConfig.custom() + .setConnectTimeout(connectTimeout) + .setConnectionRequestTimeout(connectTimeout) + .setSocketTimeout(socketTimeout) + .build()) + .addInterceptorLast(new RequestContent(true)) + .build(); + } +} From 4b560524e8ddc7f7aef9e743bb4c5d6812a7c5a3 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 26 Nov 2025 19:39:47 +0800 Subject: [PATCH 03/27] fix write split chunk api --- .../doris/cdcclient/CdcClientApplication.java | 6 +- .../apache/doris/cdcclient/common/Env.java | 9 +- .../cdcclient/constants/LoadConstants.java | 7 +- .../controller/ClientController.java | 27 +- .../{req => request}/FetchRecordReq.java | 4 +- .../model/request/FetchTableSplitsReq.java | 30 ++ .../model/request/JobBaseRecordReq.java | 35 ++ .../WriteRecordReq.java} | 24 +- .../cdcclient/model/resp/WriteMetaResp.java | 9 - .../{resp => response}/RecordWithMeta.java | 2 +- .../WriteMetaResp.java} | 9 +- .../service/PipelineCoordinator.java | 144 +++-- .../cdcclient/sink/BatchRecordBuffer.java | 12 +- .../cdcclient/sink/DorisBatchStreamLoad.java | 73 +-- .../deserialize/DebeziumJsonDeserializer.java | 21 +- .../source/factory/SourceReaderFactory.java | 17 + .../cdcclient/source/reader/SourceReader.java | 59 +- .../source/reader/SplitReadResult.java | 24 +- .../reader/mysql/MySqlSourceReader.java | 509 ++++++------------ .../source/split/AbstractSourceSplit.java | 40 +- .../cdcclient/source/split/BinlogSplit.java | 38 +- .../cdcclient/source/split/SnapshotSplit.java | 135 ++--- .../cdcclient/source/split/SourceSplit.java | 17 + .../doris/cdcclient/utils/ConfigUtil.java | 20 +- .../doris/cdcclient/utils/HttpUtil.java | 17 + 25 files changed, 608 insertions(+), 680 deletions(-) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/{req => request}/FetchRecordReq.java (91%) create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/{req/BaseRecordReq.java => request/WriteRecordReq.java} (74%) delete mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/WriteMetaResp.java rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/{resp => response}/RecordWithMeta.java (95%) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/{req/WriteRecordReq.java => response/WriteMetaResp.java} (83%) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java index 1f62cf1f56527d..94f2a4dc8c716c 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java @@ -17,8 +17,8 @@ package org.apache.doris.cdcclient; -import java.util.Arrays; import org.apache.doris.cdcclient.common.Env; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.boot.SpringApplication; @@ -27,6 +27,8 @@ import org.springframework.boot.web.servlet.ServletComponentScan; import org.springframework.boot.web.servlet.support.SpringBootServletInitializer; +import java.util.Arrays; + @SpringBootApplication @EnableConfigurationProperties @ServletComponentScan @@ -35,7 +37,7 @@ public class CdcClientApplication extends SpringBootServletInitializer { public static void main(String[] args) { LOG.info("args: {}, log.path: {}", Arrays.asList(args), System.getProperty("log.path")); - Env.getCurrentEnv().setBeHttpPort(8040); + Env.getCurrentEnv().setBackendHostPort("10.16.10.6:28747"); SpringApplication.run(CdcClientApplication.class, args); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java index 147c3045b93c1c..685e408b864721 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java @@ -23,6 +23,7 @@ import java.util.concurrent.ConcurrentHashMap; import lombok.Getter; import lombok.Setter; +import org.apache.doris.cdcclient.model.JobConfig; import org.apache.doris.cdcclient.source.factory.DataSource; import org.apache.doris.cdcclient.source.factory.SourceReaderFactory; import org.apache.doris.cdcclient.source.reader.SourceReader; @@ -30,7 +31,7 @@ public class Env { private static volatile Env INSTANCE; private final Map jobContexts; - @Getter @Setter private int beHttpPort; + @Getter @Setter private String backendHostPort; private Env() { this.jobContexts = new ConcurrentHashMap<>(); @@ -47,10 +48,10 @@ public static Env getCurrentEnv() { return INSTANCE; } - public SourceReader getReader(Long jobId, String dataSource, Map config) { - DataSource ds = resolveDataSource(dataSource); + public SourceReader getReader(JobConfig jobConfig) { + DataSource ds = resolveDataSource(jobConfig.getDataSource()); Env manager = Env.getCurrentEnv(); - return manager.getOrCreateReader(jobId, ds, config); + return manager.getOrCreateReader(jobConfig.getJobId(), ds, jobConfig.getConfig()); } private DataSource resolveDataSource(String source) { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java index 6e0f744e262ccb..b0b2f000dd09cc 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java @@ -23,7 +23,8 @@ public class LoadConstants { public static final String PORT = "port"; public static final String USERNAME = "username"; public static final String PASSWORD = "password"; - public static final String DATABASE_NAME = "database"; - public static final String INCLUDE_TABLES_LIST = "include_tables"; - public static final String EXCLUDE_TABLES_LIST = "exclude_tables"; + public static final String DATABASE = "database"; + public static final String INCLUDE_TABLES = "include_tables"; + public static final String EXCLUDE_TABLES = "exclude_tables"; + public static final String SPLIT_SIZE = "split_size"; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index 4b56fda55f244a..0671e41941b15b 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -17,16 +17,16 @@ package org.apache.doris.cdcclient.controller; +import java.util.List; import org.apache.doris.cdcclient.common.Env; -import org.apache.doris.cdcclient.model.JobConfig; -import org.apache.doris.cdcclient.model.req.FetchRecordReq; -import org.apache.doris.cdcclient.model.req.WriteRecordReq; -import org.apache.doris.cdcclient.model.resp.RecordWithMeta; -import org.apache.doris.cdcclient.model.resp.WriteMetaResp; +import org.apache.doris.cdcclient.model.request.FetchRecordReq; +import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; +import org.apache.doris.cdcclient.model.request.WriteRecordReq; +import org.apache.doris.cdcclient.model.response.RecordWithMeta; +import org.apache.doris.cdcclient.model.response.WriteMetaResp; import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; import org.apache.doris.cdcclient.service.PipelineCoordinator; import org.apache.doris.cdcclient.source.reader.SourceReader; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -36,8 +36,6 @@ import org.springframework.web.bind.annotation.RequestMethod; import org.springframework.web.bind.annotation.RestController; -import java.util.List; - @RestController public class ClientController { private static final Logger LOG = LoggerFactory.getLogger(ClientController.class); @@ -46,16 +44,13 @@ public class ClientController { /** Fetch source splits for snapshot */ @RequestMapping(path = "/api/fetchSplits", method = RequestMethod.POST) - public Object fetchSplits(@RequestBody JobConfig config) throws Exception { + public Object fetchSplits(@RequestBody FetchTableSplitsReq ftsReq) { try { - SourceReader reader = - Env.getCurrentEnv() - .getReader( - config.getJobId(), config.getDataSource(), config.getConfig()); - List splits = reader.getSourceSplits(config); + SourceReader reader = Env.getCurrentEnv().getReader(ftsReq); + List splits = reader.getSourceSplits(ftsReq); return ResponseEntityBuilder.ok(splits); } catch (IllegalArgumentException ex) { - LOG.error("Failed to fetch splits, jobId={}", config.getJobId(), ex); + LOG.error("Failed to fetch splits, jobId={}", ftsReq.getJobId(), ex); return ResponseEntityBuilder.badRequest(ex.getMessage()); } } @@ -76,7 +71,7 @@ public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { @RequestMapping(path = "/api/writeRecords", method = RequestMethod.POST) public Object writeRecord(@RequestBody WriteRecordReq recordReq) { try { - WriteMetaResp response = pipelineCoordinator.readAndWrite(recordReq); + WriteMetaResp response = pipelineCoordinator.writeRecords(recordReq); return ResponseEntityBuilder.ok(response); } catch (Exception ex) { LOG.error("Failed to write record, jobId={}", recordReq.getJobId(), ex); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/FetchRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchRecordReq.java similarity index 91% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/FetchRecordReq.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchRecordReq.java index 2a829fab29b1c0..6decb34171cac8 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/FetchRecordReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchRecordReq.java @@ -15,14 +15,14 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model.req; +package org.apache.doris.cdcclient.model.request; import lombok.Data; import lombok.EqualsAndHashCode; @Data @EqualsAndHashCode(callSuper = true) -public class FetchRecordReq extends BaseRecordReq { +public class FetchRecordReq extends JobBaseRecordReq { private boolean reload; private int fetchSize; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java new file mode 100644 index 00000000000000..eaa5106f194e32 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java @@ -0,0 +1,30 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model.request; + +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.Setter; +import org.apache.doris.cdcclient.model.JobConfig; + +@Getter +@Setter +@EqualsAndHashCode(callSuper = true) +public class FetchTableSplitsReq extends JobConfig { + private String snapshotTable; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java new file mode 100644 index 00000000000000..6696fe0bcae2a3 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model.request; + +import java.util.Map; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.Setter; +import org.apache.doris.cdcclient.model.JobConfig; + +@Getter +@Setter +@EqualsAndHashCode(callSuper = true) +public abstract class JobBaseRecordReq extends JobConfig { + protected Map meta; + + public abstract boolean isReload(); + + public abstract int getFetchSize(); +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/BaseRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java similarity index 74% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/BaseRecordReq.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java index 545756ea74066d..7550876da12b92 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/BaseRecordReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java @@ -15,25 +15,23 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model.req; +package org.apache.doris.cdcclient.model.request; -import lombok.Getter; -import lombok.Setter; +import lombok.Data; +import lombok.EqualsAndHashCode; -import java.util.Map; - -@Getter -@Setter -public abstract class BaseRecordReq { - private long jobId; - private String dataSource; - private Map meta; - private Map config; +@Data +@EqualsAndHashCode(callSuper = true) +public class WriteRecordReq extends JobBaseRecordReq { + private long maxInterval; + private String targetDatabase; + @Override public boolean isReload() { - return false; + return true; } + @Override public int getFetchSize() { return Integer.MAX_VALUE; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/WriteMetaResp.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/WriteMetaResp.java deleted file mode 100644 index 4004611c8bde64..00000000000000 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/WriteMetaResp.java +++ /dev/null @@ -1,9 +0,0 @@ -package org.apache.doris.cdcclient.model.resp; - -import java.util.Map; -import lombok.Data; - -@Data -public class WriteMetaResp { - private Map meta; -} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/RecordWithMeta.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java similarity index 95% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/RecordWithMeta.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java index f4bc38ea0615ef..621138ee34a81e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/resp/RecordWithMeta.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model.resp; +package org.apache.doris.cdcclient.model.response; import java.util.ArrayList; import java.util.HashMap; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/WriteRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/WriteMetaResp.java similarity index 83% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/WriteRecordReq.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/WriteMetaResp.java index 1f59cfdf9795aa..9305e1064717c0 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/req/WriteRecordReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/WriteMetaResp.java @@ -15,11 +15,12 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model.req; +package org.apache.doris.cdcclient.model.response; +import java.util.Map; import lombok.Data; -import lombok.EqualsAndHashCode; @Data -@EqualsAndHashCode(callSuper = true) -public class WriteRecordReq extends BaseRecordReq {} +public class WriteMetaResp { + private Map meta; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 51d9f79511658f..c4a875f94756cb 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -17,20 +17,25 @@ package org.apache.doris.cdcclient.service; +import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; + +import io.debezium.data.Envelope; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.commons.collections.CollectionUtils; import org.apache.doris.cdcclient.common.Env; -import org.apache.doris.cdcclient.model.JobConfig; -import org.apache.doris.cdcclient.model.req.FetchRecordReq; -import org.apache.doris.cdcclient.model.req.WriteRecordReq; -import org.apache.doris.cdcclient.model.resp.RecordWithMeta; -import org.apache.doris.cdcclient.model.resp.WriteMetaResp; +import org.apache.doris.cdcclient.model.request.FetchRecordReq; +import org.apache.doris.cdcclient.model.request.WriteRecordReq; +import org.apache.doris.cdcclient.model.response.RecordWithMeta; +import org.apache.doris.cdcclient.model.response.WriteMetaResp; import org.apache.doris.cdcclient.sink.DorisBatchStreamLoad; import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; import org.apache.doris.cdcclient.source.reader.SourceReader; import org.apache.doris.cdcclient.source.reader.SplitReadResult; - -import io.debezium.data.Envelope; -import org.apache.commons.collections.CollectionUtils; import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; @@ -38,12 +43,6 @@ import org.slf4j.LoggerFactory; import org.springframework.stereotype.Component; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - /** Pipeline coordinator. */ @Component public class PipelineCoordinator { @@ -61,72 +60,92 @@ public PipelineCoordinator() { /** Read data from SourceReader and return it with meta information. */ public RecordWithMeta read(FetchRecordReq recordReq) throws Exception { - SourceReader reader = - Env.getCurrentEnv() - .getReader( - recordReq.getJobId(), - recordReq.getDataSource(), - recordReq.getConfig()); + SourceReader reader = Env.getCurrentEnv().getReader(recordReq); return reader.read(recordReq); } /** Read data from SourceReader and write it to Doris, while returning meta information. */ - public WriteMetaResp readAndWrite(WriteRecordReq writeRecordReq) throws Exception { - SourceReader sourceReader = - Env.getCurrentEnv() - .getReader( - writeRecordReq.getJobId(), - writeRecordReq.getDataSource(), - writeRecordReq.getConfig()); - JobConfig jobConfig = - new JobConfig( - writeRecordReq.getJobId(), - writeRecordReq.getDataSource(), - writeRecordReq.getConfig()); - - Map offsetMeta = writeRecordReq.getMeta(); + public WriteMetaResp writeRecords(WriteRecordReq writeRecordReq) throws Exception { + SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordReq); WriteMetaResp recordResponse = new WriteMetaResp(); - SplitReadResult readResult = sourceReader.readSplitRecords(writeRecordReq); - if (readResult == null || readResult.isEmpty()) { - setDefaultMeta(recordResponse, offsetMeta, readResult); - return recordResponse; - } - DorisBatchStreamLoad batchStreamLoad = getOrCreateBatchStreamLoad(writeRecordReq.getJobId()); boolean readBinlog = readResult.isReadBinlog(); boolean pureBinlogPhase = readResult.isPureBinlogPhase(); - // 直接使用迭代器,边读边写 + boolean hasData = false; + // Record start time for maxInterval check + long startTime = System.currentTimeMillis(); + long maxIntervalMillis = writeRecordReq.getMaxInterval() * 1000; + + // Use iterators to read and write. Iterator iterator = readResult.getRecordIterator(); while (iterator != null && iterator.hasNext()) { SourceRecord element = iterator.next(); if (RecordUtils.isDataChangeRecord(element)) { List serializedRecords = - serializer.deserialize(jobConfig.getConfig(), element); + serializer.deserialize(writeRecordReq.getConfig(), element); if (!CollectionUtils.isEmpty(serializedRecords)) { - String database = "doris_cdc"; // doris database + String database = writeRecordReq.getTargetDatabase(); String table = extractTable(element); - + hasData = true; for (String record : serializedRecords) { batchStreamLoad.writeRecord(database, table, record.getBytes()); } Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); - if (readBinlog && readResult.getSplitId() != null) { - lastMeta.put(SPLIT_ID, readResult.getSplitId()); + if (readBinlog && sourceReader.getSplitId(readResult.getSplit()) != null) { + lastMeta.put(SPLIT_ID, sourceReader.getSplitId(readResult.getSplit())); lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); } recordResponse.setMeta(lastMeta); } } + // Check if maxInterval has been exceeded + long elapsedTime = System.currentTimeMillis() - startTime; + if (maxIntervalMillis > 0 && elapsedTime >= maxIntervalMillis) { + LOG.info( + "Max interval {} seconds reached, stopping data reading", + writeRecordReq.getMaxInterval()); + break; + } + } + if (hasData) { + // wait all stream load finish + batchStreamLoad.forceFlush(); + } + + sourceReader.finishSplitRecords(); + // update offset meta + if (!readBinlog) { + Map offsetRes = + sourceReader.extractSnapshotOffset( + readResult.getSplitState(), readResult.getSplit()); + if (offsetRes != null) { + recordResponse.setMeta(offsetRes); + } } - // wait stream load finish - batchStreamLoad.forceFlush(); + if (!hasData) { + if (readBinlog) { + Map offsetRes = + sourceReader.extractBinlogOffset(readResult.getSplit(), pureBinlogPhase); + if (offsetRes != null) { + recordResponse.setMeta(offsetRes); + } else { + // Fallback to request meta if extraction fails + Map fallbackOffset = new HashMap<>(writeRecordReq.getMeta()); + fallbackOffset.put(SPLIT_ID, BINLOG_SPLIT_ID); + fallbackOffset.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + recordResponse.setMeta(fallbackOffset); + } + } else { + recordResponse.setMeta(writeRecordReq.getMeta()); + } + } return recordResponse; } @@ -135,7 +154,7 @@ private DorisBatchStreamLoad getOrCreateBatchStreamLoad(Long jobId) { jobId, k -> { LOG.info("Create DorisBatchStreamLoad for jobId={}", jobId); - return new DorisBatchStreamLoad(); + return new DorisBatchStreamLoad(jobId); }); } @@ -147,33 +166,6 @@ public void closeJob(Long jobId) { } } - private void setDefaultMeta( - WriteMetaResp recordResponse, - Map offsetMeta, - SplitReadResult readResult) { - if (readResult == null) { - recordResponse.setMeta(offsetMeta); - return; - } - - boolean readBinlog = readResult.isReadBinlog(); - if (readBinlog) { - Map offsetRes; - if (readResult.getDefaultOffset() != null) { - offsetRes = new HashMap<>(readResult.getDefaultOffset()); - } else { - offsetRes = new HashMap<>(offsetMeta); - } - if (readResult.getSplitId() != null) { - offsetRes.put(SPLIT_ID, readResult.getSplitId()); - } - offsetRes.put(PURE_BINLOG_PHASE, String.valueOf(readResult.isPureBinlogPhase())); - recordResponse.setMeta(offsetRes); - } else { - recordResponse.setMeta(offsetMeta); - } - } - private String extractTable(SourceRecord record) { Struct value = (Struct) record.value(); return value.getStruct(Envelope.FieldName.SOURCE).getString("table"); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java index 21ddf4bf82fa60..61beb8eb9c43ef 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java @@ -33,20 +33,17 @@ public class BatchRecordBuffer { private boolean loadBatchFirstRecord = true; private String database; private String table; - private final long createTime = System.currentTimeMillis(); - private long retainTime = 0; public BatchRecordBuffer() { this.buffer = new LinkedList<>(); } - public BatchRecordBuffer(String database, String table, byte[] lineDelimiter, long retainTime) { + public BatchRecordBuffer(String database, String table, byte[] lineDelimiter) { super(); this.database = database; this.table = table; this.lineDelimiter = lineDelimiter; this.buffer = new LinkedList<>(); - this.retainTime = retainTime; } public int insert(byte[] record) { @@ -145,11 +142,4 @@ public String getTableIdentifier() { public byte[] getLineDelimiter() { return lineDelimiter; } - - public boolean shouldFlush() { - // When the buffer create time is later than the first interval trigger, - // the write will not be triggered in the next interval, - // so multiply it by 1.5 to trigger it as early as possible. - return (System.currentTimeMillis() - createTime) * 1.5 > retainTime; - } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java index 42fcef33d7958e..de2eaf6ef19143 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java @@ -45,6 +45,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.commons.codec.binary.Base64; import org.apache.commons.lang3.StringUtils; +import org.apache.doris.cdcclient.common.Env; import org.apache.doris.cdcclient.exception.StreamLoadException; import org.apache.doris.cdcclient.utils.HttpUtil; import org.apache.flink.annotation.VisibleForTesting; @@ -63,12 +64,11 @@ public class DorisBatchStreamLoad implements Serializable { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final List DORIS_SUCCESS_STATUS = new ArrayList<>(Arrays.asList("Success")); - private static final long STREAM_LOAD_MAX_BYTES = 1 * 1024 * 1024 * 1024L; // 1 GB - private static final long STREAM_LOAD_MAX_ROWS = Integer.MAX_VALUE; + private final int FLUSH_QUEUE_SIZE = 1; + private final long STREAM_LOAD_MAX_BYTES = 500 * 1024 * 1024L; // 500MB + private final int RETRY = 3; private final byte[] lineDelimiter = "\n".getBytes(); - ; private static final String LOAD_URL_PATTERN = "http://%s/api/%s/%s/_stream_load"; - private String loadUrl; private String hostPort; private Map bufferMap = new ConcurrentHashMap<>(); private ExecutorService loadExecutorService; @@ -82,16 +82,12 @@ public class DorisBatchStreamLoad implements Serializable { private final Lock lock = new ReentrantLock(); private final Condition block = lock.newCondition(); private final Map bufferMapLock = new ConcurrentHashMap<>(); - private int FLUSH_QUEUE_SIZE = 1; - private int FLUSH_MAX_BYTE_SIZE = 100 * 1024 * 1024; - private int FLUSH_INTERVAL_MS = 10 * 1000; - private int RETRY = 3; - public DorisBatchStreamLoad() { - this.hostPort = "10.16.10.6:28747"; + public DorisBatchStreamLoad(long jobId) { + this.hostPort = Env.getCurrentEnv().getBackendHostPort(); this.flushQueue = new LinkedBlockingDeque<>(1); - // maxBlockedBytes ensures that a buffer can be written even if the queue is full - this.maxBlockedBytes = (long) FLUSH_MAX_BYTE_SIZE * (FLUSH_QUEUE_SIZE + 1); + // maxBlockedBytes is two times of FLUSH_MAX_BYTE_SIZE + this.maxBlockedBytes = STREAM_LOAD_MAX_BYTES * 2; this.loadAsyncExecutor = new LoadAsyncExecutor(FLUSH_QUEUE_SIZE); this.loadExecutorService = new ThreadPoolExecutor( @@ -100,7 +96,7 @@ public DorisBatchStreamLoad() { 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(1), - new DefaultThreadFactory("stream-load-executor"), + new DefaultThreadFactory("stream-load-executor-" + jobId), new ThreadPoolExecutor.AbortPolicy()); this.started = new AtomicBoolean(true); this.loadExecutorService.execute(loadAsyncExecutor); @@ -119,16 +115,14 @@ public void writeRecord(String database, String table, byte[] record) { getLock(bufferKey).readLock().lock(); BatchRecordBuffer buffer = bufferMap.computeIfAbsent( - bufferKey, - k -> - new BatchRecordBuffer( - database, table, this.lineDelimiter, FLUSH_INTERVAL_MS)); + bufferKey, k -> new BatchRecordBuffer(database, table, this.lineDelimiter)); int bytes = buffer.insert(record); currentCacheBytes.addAndGet(bytes); getLock(bufferKey).readLock().unlock(); if (currentCacheBytes.get() > maxBlockedBytes) { + cacheFullFlush(); lock.lock(); try { while (currentCacheBytes.get() >= maxBlockedBytes) { @@ -148,48 +142,37 @@ public void writeRecord(String database, String table, byte[] record) { } } + public boolean cacheFullFlush() { + return doFlush(true, true); + } + public boolean forceFlush() { - return doFlush(null, true, false); + return doFlush(true, false); } - private synchronized boolean doFlush( - String bufferKey, boolean waitUtilDone, boolean bufferFull) { + private synchronized boolean doFlush(boolean waitUtilDone, boolean cacheFull) { checkFlushException(); - if (waitUtilDone || bufferFull) { - boolean flush = flush(bufferKey, waitUtilDone); - return flush; - } else if (flushQueue.size() < FLUSH_QUEUE_SIZE) { - boolean flush = flush(bufferKey, false); - return flush; + if (waitUtilDone || cacheFull) { + return flush(waitUtilDone); } return false; } - private synchronized boolean flush(String bufferKey, boolean waitUtilDone) { + private synchronized boolean flush(boolean waitUtilDone) { if (!waitUtilDone && bufferMap.isEmpty()) { // bufferMap may have been flushed by other threads - LOG.info("bufferMap is empty, no need to flush {}", bufferKey); + LOG.info("bufferMap is empty, no need to flush"); return false; } - if (null == bufferKey) { - boolean flush = false; - for (String key : bufferMap.keySet()) { - BatchRecordBuffer buffer = bufferMap.get(key); - if (waitUtilDone || buffer.shouldFlush()) { - // Ensure that the interval satisfies intervalMS - flushBuffer(key); - flush = true; - } - } - if (!waitUtilDone && !flush) { - return false; + for (String key : bufferMap.keySet()) { + if (waitUtilDone) { + // Ensure that the interval satisfies intervalMS + flushBuffer(key); } - } else if (bufferMap.containsKey(bufferKey)) { - flushBuffer(bufferKey); - } else { - LOG.warn("buffer not found for key: {}, may be already flushed.", bufferKey); } - if (waitUtilDone) { + if (!waitUtilDone) { + return false; + } else { waitAsyncLoadFinish(); } return true; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java index 76edb7998d0189..e28d61ca90a778 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java @@ -27,17 +27,6 @@ import io.debezium.time.NanoTimestamp; import io.debezium.time.Timestamp; import io.debezium.time.ZonedTimestamp; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; -import org.apache.flink.cdc.debezium.utils.TemporalConversions; -import org.apache.flink.table.data.TimestampData; -import org.apache.kafka.connect.data.Decimal; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.source.SourceRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.math.BigDecimal; import java.time.Instant; @@ -48,6 +37,16 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; +import org.apache.flink.cdc.debezium.utils.TemporalConversions; +import org.apache.flink.table.data.TimestampData; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DebeziumJsonDeserializer implements SourceRecordDeserializer> { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java index f59d11510aaeb3..507e949498de80 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.cdcclient.source.factory; import java.util.Map; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index d9450094a527ce..1428c579d2abb7 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -17,53 +17,54 @@ package org.apache.doris.cdcclient.source.reader; -import com.fasterxml.jackson.core.JsonProcessingException; import java.util.List; -import org.apache.doris.cdcclient.model.JobConfig; -import org.apache.doris.cdcclient.model.req.BaseRecordReq; -import org.apache.doris.cdcclient.model.req.FetchRecordReq; -import org.apache.doris.cdcclient.model.resp.RecordWithMeta; +import java.util.Map; +import org.apache.doris.cdcclient.model.request.FetchRecordReq; +import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; +import org.apache.doris.cdcclient.model.request.JobBaseRecordReq; +import org.apache.doris.cdcclient.model.response.RecordWithMeta; import org.apache.doris.cdcclient.source.split.SourceSplit; /** - * SourceReader 接口,支持泛型以指定 Split 和 SplitState 类型。 + * SourceReader interface * - * @param Split 类型(如 MySqlSplit) - * @param SplitState 类型(如 MySqlSplitState) + * @param Split (MySqlSplit) + * @param SplitState(MySqlSplitState) */ public interface SourceReader { - /** Initialization, called when the program starts */ void initialize(); - /** - * Divide the data to be read. For example: split mysql to chunks - * - * @return - */ - List getSourceSplits(JobConfig config) throws JsonProcessingException; + /** Divide the data to be read. For example: split mysql to chunks */ + List getSourceSplits(FetchTableSplitsReq config); - /** - * Reading Data - * - * @param meta - * @return - * @throws Exception - */ + /** Reading Data */ RecordWithMeta read(FetchRecordReq meta) throws Exception; - /** - * Reading Data for split reader - * - * @param baseReq 基础请求 - * @return 读取结果,包含 SourceRecord 列表和状态信息 - */ - default SplitReadResult readSplitRecords(BaseRecordReq baseReq) + /** Reading Data for split reader */ + default SplitReadResult readSplitRecords(JobBaseRecordReq baseReq) throws Exception { throw new UnsupportedOperationException( "readSplitRecords is not supported by " + this.getClass().getName()); } + /** Extract offset information from snapshot split state. */ + Map extractSnapshotOffset(Object splitState, Object split); + + /** Extract offset information from binlog split. */ + Map extractBinlogOffset(Object split, boolean pureBinlogPhase); + + /** + * Get split ID from the split. This method should be implemented by each SourceReader to handle + * its specific Split type. + * + * @param split the split + * @return split ID, or null if split is null + */ + String getSplitId(Object split); + /** Called when closing */ void close(Long jobId); + + void finishSplitRecords(); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java index f1c06ead7c637f..b3f9d06edb6ffb 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java @@ -18,29 +18,21 @@ package org.apache.doris.cdcclient.source.reader; import java.util.Iterator; -import java.util.Map; import lombok.Data; import org.apache.kafka.connect.source.SourceRecord; /** - * 保存从 split 中读取的结果。 使用泛型支持不同类型的 Split 和 SplitState。 使用迭代器模式,避免将所有数据加载到内存。 + * The result of reading a split with iterator. * - * @param Split 类型(如 MySqlSplit) - * @param SplitState 类型(如 MySqlSplitState) + * @param Split type (MySqlSplit) + * @param SplitState type (MySqlSplitState) */ @Data public class SplitReadResult { - private Iterator recordIterator; // 使用迭代器,支持流式处理 - private Map lastMeta; - private SplitState splitState; // Split state,由具体的 reader 管理 - private boolean readBinlog; // 是否读取 binlog(对于支持 binlog 的数据源) - private boolean pureBinlogPhase; // 是否处于纯 binlog 阶段 - private Split split; // Split,由具体的 reader 管理 - private String splitId; // split ID - private Map defaultOffset; // 默认的 offset(用于没有数据时) + private Iterator recordIterator; + private SplitState splitState; + private Split split; - /** 检查是否有数据(延迟检查,因为迭代器可能还没有被消费) */ - public boolean isEmpty() { - return recordIterator == null || !recordIterator.hasNext(); - } + private boolean readBinlog; + private boolean pureBinlogPhase; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index ac8b77abaf5ad0..edbf2598d98fb6 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -17,20 +17,7 @@ package org.apache.doris.cdcclient.source.reader.mysql; -import org.apache.doris.cdcclient.constants.LoadConstants; -import org.apache.doris.cdcclient.model.JobConfig; -import org.apache.doris.cdcclient.model.req.BaseRecordReq; -import org.apache.doris.cdcclient.model.req.FetchRecordReq; -import org.apache.doris.cdcclient.model.resp.RecordWithMeta; -import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; -import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; -import org.apache.doris.cdcclient.source.reader.SourceReader; -import org.apache.doris.cdcclient.source.reader.SplitReadResult; -import org.apache.doris.cdcclient.source.reader.SplitRecords; -import org.apache.doris.cdcclient.source.split.AbstractSourceSplit; -import org.apache.doris.cdcclient.source.split.BinlogSplit; -import org.apache.doris.cdcclient.source.split.SnapshotSplit; -import org.apache.doris.cdcclient.utils.ConfigUtil; +import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; @@ -43,7 +30,33 @@ import io.debezium.relational.TableId; import io.debezium.relational.history.HistoryRecord; import io.debezium.relational.history.TableChanges; +import java.io.IOException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.collections.CollectionUtils; +import org.apache.doris.cdcclient.constants.LoadConstants; +import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.cdcclient.model.request.FetchRecordReq; +import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; +import org.apache.doris.cdcclient.model.request.JobBaseRecordReq; +import org.apache.doris.cdcclient.model.response.RecordWithMeta; +import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; +import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; +import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.cdcclient.source.reader.SplitReadResult; +import org.apache.doris.cdcclient.source.reader.SplitRecords; +import org.apache.doris.cdcclient.source.split.AbstractSourceSplit; +import org.apache.doris.cdcclient.source.split.BinlogSplit; +import org.apache.doris.cdcclient.source.split.SnapshotSplit; +import org.apache.doris.cdcclient.utils.ConfigUtil; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.utils.Preconditions; import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; @@ -51,7 +64,6 @@ import org.apache.flink.cdc.connectors.mysql.debezium.reader.DebeziumReader; import org.apache.flink.cdc.connectors.mysql.debezium.reader.SnapshotSplitReader; import org.apache.flink.cdc.connectors.mysql.debezium.task.context.StatefulTaskContext; -import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; import org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlSnapshotSplitAssigner; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; @@ -72,52 +84,34 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Objects; -import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; -import java.util.stream.Collectors; - public class MySqlSourceReader implements SourceReader { private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceReader.class); private static ObjectMapper objectMapper = new ObjectMapper(); private static final String SPLIT_ID = "splitId"; private static final String FINISH_SPLITS = "finishSplits"; - private static final String ASSIGNED_SPLITS = "assignedSplits"; - private static final String SNAPSHOT_TABLE = "snapshotTable"; private static final String PURE_BINLOG_PHASE = "pureBinlogPhase"; private static final FlinkJsonTableChangeSerializer TABLE_CHANGE_SERIALIZER = new FlinkJsonTableChangeSerializer(); private SourceRecordDeserializer> serializer; - private Map jobContexts; + private JobRuntimeContext jobRuntimeContext; public MySqlSourceReader() { this.serializer = new DebeziumJsonDeserializer(); - this.jobContexts = new ConcurrentHashMap<>(); + this.jobRuntimeContext = new JobRuntimeContext(); } @Override public void initialize() {} @Override - public List getSourceSplits(JobConfig config) - throws JsonProcessingException { - MySqlSourceConfig sourceConfig = getSourceConfig(config); + public List getSourceSplits(FetchTableSplitsReq ftsReq) { + MySqlSourceConfig sourceConfig = getSourceConfig(ftsReq); StartupMode startupMode = sourceConfig.getStartupOptions().startupMode; List remainingSnapshotSplits = new ArrayList<>(); MySqlBinlogSplit remainingBinlogSplit = null; if (startupMode.equals(StartupMode.INITIAL)) { - String snapshotTable = config.getConfig().get(SNAPSHOT_TABLE); remainingSnapshotSplits = - startSplitChunks(sourceConfig, snapshotTable, config.getConfig()); + startSplitChunks(sourceConfig, ftsReq.getSnapshotTable(), ftsReq.getConfig()); } else { remainingBinlogSplit = new MySqlBinlogSplit( @@ -133,15 +127,9 @@ public List getSourceSplits(JobConfig config) for (MySqlSnapshotSplit snapshotSplit : remainingSnapshotSplits) { String splitId = snapshotSplit.splitId(); String tableId = snapshotSplit.getTableId().identifier(); - String splitStart = - snapshotSplit.getSplitStart() == null - ? null - : objectMapper.writeValueAsString(snapshotSplit.getSplitStart()); - String splitEnd = - snapshotSplit.getSplitEnd() == null - ? null - : objectMapper.writeValueAsString(snapshotSplit.getSplitEnd()); - String splitKey = snapshotSplit.getSplitKeyType().getFieldNames().get(0); + Object[] splitStart = snapshotSplit.getSplitStart(); + Object[] splitEnd = snapshotSplit.getSplitEnd(); + List splitKey = snapshotSplit.getSplitKeyType().getFieldNames(); SnapshotSplit split = new SnapshotSplit(splitId, tableId, splitKey, splitStart, splitEnd, null); splits.add(split); @@ -156,180 +144,99 @@ public List getSourceSplits(JobConfig config) } /** - * 1. If the SplitRecords iterator has it, read the iterator directly. - * 2. If there is a binlogreader, poll it. - * 3. If there is none, resubmit split. 4. If reload is true, need to - * reset binlogSplitReader and submit split. + * 1. If the SplitRecords iterator has it, read the iterator directly. 2. If there is a + * binlogreader, poll it. 3. If there is none, resubmit split. 4. If reload is true, need to + * reset binlogSplitReader and submit split. */ @Override public RecordWithMeta read(FetchRecordReq fetchRecord) throws Exception { SplitReadResult readResult = readSplitRecords(fetchRecord); - - JobConfig jobConfig = - new JobConfig( - fetchRecord.getJobId(), - fetchRecord.getDataSource(), - fetchRecord.getConfig()); - Map offsetMeta = fetchRecord.getMeta(); - RecordWithMeta recordResponse = new RecordWithMeta(); - - return buildRecordResponse(fetchRecord, jobConfig, offsetMeta, recordResponse, readResult); + return buildRecordResponse(fetchRecord, readResult); } - /** - * Prepare split information based on current context: - * - If there is an active split being consumed, reuse it directly; - * - Otherwise, create a new snapshot/binlog split based on offset and start the reader. - */ - private SplitProcessingContext prepareSplitProcessingContext( - JobRuntimeContext jobRuntimeContext, - JobConfig jobConfig, - Map offsetMeta) + /** read split records. */ + @Override + public SplitReadResult readSplitRecords(JobBaseRecordReq baseReq) throws Exception { - SplitProcessingContext context = new SplitProcessingContext(); + Map offsetMeta = baseReq.getMeta(); + if (offsetMeta == null || offsetMeta.isEmpty()) { + throw new RuntimeException("miss meta offset"); + } + + // If there is an active split being consumed, reuse it directly; + // Otherwise, create a new snapshot/binlog split based on offset and start the reader. + boolean readBinlog = true; + MySqlSplit split = null; + boolean pureBinlogPhase = true; SplitRecords currentSplitRecords = jobRuntimeContext.getCurrentSplitRecords(); if (currentSplitRecords == null) { DebeziumReader currentReader = jobRuntimeContext.getCurrentReader(); if (currentReader instanceof BinlogSplitReader) { + // only for binlog reader currentSplitRecords = pollSplitRecordsWithCurrentReader(currentReader); } else if (currentReader == null) { - Tuple2 splitFlag = createMySqlSplit(offsetMeta, jobConfig); - context.setSplit(splitFlag.f0); - context.setPureBinlogPhase(splitFlag.f1); - context.setReadBinlog(!context.getSplit().isSnapshotSplit()); - LOG.info( - "Job {} submitted new split {}, readBinlog={}", - jobConfig.getJobId(), - context.getSplit().splitId(), - context.isReadBinlog()); - currentSplitRecords = pollSplitRecordsWithSplit(context.getSplit(), jobConfig); + // build split + Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); + split = splitFlag.f0; + pureBinlogPhase = splitFlag.f1; + readBinlog = !split.isSnapshotSplit(); + currentSplitRecords = pollSplitRecordsWithSplit(split, baseReq); + jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); } else { - throw new IllegalStateException( - String.format( - "Unsupported reader type %s for job %d", - currentReader.getClass().getName(), jobConfig.getJobId())); + throw new RuntimeException("Should not happen"); } - jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); } - context.setSplitRecords(currentSplitRecords); - return context; - } - /** - * When a client requests a reload, the split is reconstructed and the binlog reader is reset. - */ - private SplitProcessingContext reloadSplitProcessingContext( - JobRuntimeContext jobRuntimeContext, - JobConfig jobConfig, - Map offsetMeta) - throws Exception { - SplitProcessingContext context = new SplitProcessingContext(); - Tuple2 splitFlag = createMySqlSplit(offsetMeta, jobConfig); - context.setSplit(splitFlag.f0); - context.setPureBinlogPhase(splitFlag.f1); - context.setReadBinlog(!context.getSplit().isSnapshotSplit()); - LOG.info( - "Job {} reload split {}, readBinlog={}", - jobConfig.getJobId(), - context.getSplit() == null ? "null" : context.getSplit().splitId(), - context.isReadBinlog()); - closeBinlogReader(jobConfig.getJobId()); - SplitRecords currentSplitRecords = pollSplitRecordsWithSplit(context.getSplit(), jobConfig); - jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); - context.setSplitRecords(currentSplitRecords); - return context; - } - - /** read split records. */ - @Override - public SplitReadResult readSplitRecords(BaseRecordReq baseReq) - throws Exception { - JobConfig jobConfig = - new JobConfig(baseReq.getJobId(), baseReq.getDataSource(), baseReq.getConfig()); - Map offsetMeta = baseReq.getMeta(); - if (offsetMeta == null || offsetMeta.isEmpty()) { - throw new RuntimeException("miss meta offset"); - } - - JobRuntimeContext jobRuntimeContext = getJobRuntimeContext(jobConfig.getJobId()); - SplitProcessingContext processingContext = - prepareSplitProcessingContext(jobRuntimeContext, jobConfig, offsetMeta); - - if (baseReq.isReload() && processingContext.getSplit() == null) { - processingContext = - reloadSplitProcessingContext(jobRuntimeContext, jobConfig, offsetMeta); + // When a client requests a reload, the split is reconstructed and the binlog reader is + // reset. + if (baseReq.isReload() && split == null) { + Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); + split = splitFlag.f0; + pureBinlogPhase = splitFlag.f1; + // reset binlog reader + closeBinlogReader(); + currentSplitRecords = pollSplitRecordsWithSplit(split, baseReq); + jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); } - return readSplitRecords(baseReq, jobRuntimeContext, processingContext); - } - - private SplitReadResult readSplitRecords( - BaseRecordReq baseReq, - JobRuntimeContext jobRuntimeContext, - SplitProcessingContext processingContext) { - int fetchSize = baseReq.getFetchSize(); - SplitRecords currentSplitRecords = processingContext.getSplitRecords(); - boolean readBinlog = processingContext.isReadBinlog(); - boolean pureBinlogPhase = processingContext.isPureBinlogPhase(); - MySqlSplit split = processingContext.getSplit(); - + // build response with iterator SplitReadResult result = new SplitReadResult<>(); MySqlSplitState currentSplitState = null; - if (!readBinlog && split != null) { + if (!readBinlog) { currentSplitState = new MySqlSnapshotSplitState(split.asSnapshotSplit()); } Iterator filteredIterator = - new FilteredRecordIterator( - currentSplitRecords, - currentSplitState, - fetchSize, - readBinlog, - pureBinlogPhase, - result); + new FilteredRecordIterator(currentSplitRecords, currentSplitState); result.setRecordIterator(filteredIterator); - jobRuntimeContext.setCurrentSplitRecords(null); result.setSplitState(currentSplitState); result.setReadBinlog(readBinlog); result.setPureBinlogPhase(pureBinlogPhase); result.setSplit(split); - - // set splitId and defaultOffset - if (split != null) { - result.setSplitId(split.splitId()); - if (readBinlog && split instanceof MySqlBinlogSplit) { - result.setDefaultOffset(split.asBinlogSplit().getStartingOffset().getOffset()); - } - } else if (readBinlog) { - result.setSplitId(BINLOG_SPLIT_ID); - } - return result; } /** build RecordWithMeta */ private RecordWithMeta buildRecordResponse( - FetchRecordReq fetchRecord, - JobConfig jobConfig, - Map offsetMeta, - RecordWithMeta recordResponse, - SplitReadResult readResult) + FetchRecordReq fetchRecord, SplitReadResult readResult) throws Exception { + RecordWithMeta recordResponse = new RecordWithMeta(); boolean readBinlog = readResult.isReadBinlog(); boolean pureBinlogPhase = readResult.isPureBinlogPhase(); MySqlSplit split = readResult.getSplit(); MySqlSplitState currentSplitState = readResult.getSplitState(); - + int count = 0; // Serialize records and add them to the response (collect from iterator) Iterator iterator = readResult.getRecordIterator(); while (iterator != null && iterator.hasNext()) { SourceRecord element = iterator.next(); - List serializedRecords = serializer.deserialize(jobConfig.getConfig(), element); + List serializedRecords = + serializer.deserialize(fetchRecord.getConfig(), element); if (!CollectionUtils.isEmpty(serializedRecords)) { recordResponse.getRecords().addAll(serializedRecords); - + count += serializedRecords.size(); // update meta Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); if (readBinlog) { @@ -337,9 +244,13 @@ private RecordWithMeta buildRecordResponse( lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); recordResponse.setMeta(lastMeta); } + if (count >= fetchRecord.getFetchSize()) { + return recordResponse; + } } } + finishSplitRecords(); // Set meta information if (!readBinlog && currentSplitState != null) { BinlogOffset highWatermark = @@ -350,7 +261,7 @@ private RecordWithMeta buildRecordResponse( } if (CollectionUtils.isEmpty(recordResponse.getRecords())) { if (readBinlog) { - Map offsetRes = new HashMap<>(offsetMeta); + Map offsetRes = new HashMap<>(fetchRecord.getMeta()); if (split != null) { offsetRes = split.asBinlogSplit().getStartingOffset().getOffset(); } @@ -360,11 +271,7 @@ private RecordWithMeta buildRecordResponse( } else { recordResponse.setMeta(fetchRecord.getMeta()); } - } else if (readResult.getLastMeta() != null) { - // If there is meta from the last record, use it - recordResponse.setMeta(readResult.getLastMeta()); } - return recordResponse; } @@ -379,14 +286,10 @@ private void refreshTableChanges(SourceRecord element, Long jobId) throws IOExce HistoryRecord historyRecord = RecordUtils.getHistoryRecord(element); Array tableChanges = historyRecord.document().getArray(HistoryRecord.Fields.TABLE_CHANGES); TableChanges changes = TABLE_CHANGE_SERIALIZER.deserialize(tableChanges, true); - JobRuntimeContext jobContext = jobContexts.get(jobId); - if (jobContext == null) { - return; - } - Map tableChangeMap = jobContext.getTableSchemas(); + Map tableChangeMap = jobRuntimeContext.getTableSchemas(); if (tableChangeMap == null) { tableChangeMap = new ConcurrentHashMap<>(); - jobContext.setTableSchemas(tableChangeMap); + jobRuntimeContext.setTableSchemas(tableChangeMap); } for (TableChanges.TableChange tblChange : changes) { tableChangeMap.put(tblChange.getTable().id(), tblChange); @@ -409,19 +312,17 @@ private Tuple2 createMySqlSplit( private MySqlSnapshotSplit createSnapshotSplit(Map offset, JobConfig jobConfig) throws JsonProcessingException { String splitId = offset.get(SPLIT_ID); - SnapshotSplit snapshotSplit = objectMapper.convertValue(offset, SnapshotSplit.class); + SnapshotSplit snapshotSplit = SnapshotSplit.fromMap(offset); TableId tableId = TableId.parse(snapshotSplit.getTableId()); - Object[] splitStart = - snapshotSplit.getSplitStart() == null - ? null - : objectMapper.readValue(snapshotSplit.getSplitStart(), Object[].class); - Object[] splitEnd = - snapshotSplit.getSplitEnd() == null - ? null - : objectMapper.readValue(snapshotSplit.getSplitEnd(), Object[].class); - String splitKey = snapshotSplit.getSplitKey(); + Object[] splitStart = snapshotSplit.getSplitStart(); + Object[] splitEnd = snapshotSplit.getSplitEnd(); + List splitKeys = snapshotSplit.getSplitKey(); Map tableSchemas = getTableSchemas(jobConfig); TableChanges.TableChange tableChange = tableSchemas.get(tableId); + Preconditions.checkNotNull( + tableChange, "Can not find table " + tableId + " in job " + jobConfig.getJobId()); + // only support one split key + String splitKey = splitKeys.get(0); Column splitColumn = tableChange.getTable().columnWithName(splitKey); RowType splitType = ChunkUtils.getChunkKeyColumnType(splitColumn); MySqlSnapshotSplit split = @@ -441,25 +342,21 @@ private Tuple2 createBinlogSplit( List finishedSnapshotSplitInfos = new ArrayList<>(); BinlogOffset minOffsetFinishSplits = null; BinlogOffset maxOffsetFinishSplits = null; - if (meta.containsKey(FINISH_SPLITS) && meta.containsKey(ASSIGNED_SPLITS)) { - // Construct binlogsplit based on the finished split and assigned split. + if (meta.containsKey(FINISH_SPLITS)) { // List + // Construct binlogsplit based on the finished split String finishSplitsOffset = meta.remove(FINISH_SPLITS); - String assignedSplits = meta.remove(ASSIGNED_SPLITS); - Map> splitFinishedOffsets = - objectMapper.readValue( - finishSplitsOffset, - new TypeReference>>() {}); - Map assignedSplitsMap = + List splitWithHW = objectMapper.readValue( - assignedSplits, new TypeReference>() {}); + finishSplitsOffset, new TypeReference>() {}); + List assignedSplitLists = - assignedSplitsMap.values().stream() + splitWithHW.stream() .sorted(Comparator.comparing(AbstractSourceSplit::getSplitId)) - .collect(Collectors.toList()); + .toList(); for (SnapshotSplit split : assignedSplitLists) { // find the min binlog offset - Map offsetMap = splitFinishedOffsets.get(split.getSplitId()); + Map offsetMap = split.getHighWatermark(); BinlogOffset binlogOffset = new BinlogOffset(offsetMap); if (minOffsetFinishSplits == null || binlogOffset.isBefore(minOffsetFinishSplits)) { minOffsetFinishSplits = binlogOffset; @@ -467,21 +364,12 @@ private Tuple2 createBinlogSplit( if (maxOffsetFinishSplits == null || binlogOffset.isAfter(maxOffsetFinishSplits)) { maxOffsetFinishSplits = binlogOffset; } - Object[] splitStart = - split.getSplitStart() == null - ? null - : objectMapper.readValue(split.getSplitStart(), Object[].class); - Object[] splitEnd = - split.getSplitEnd() == null - ? null - : objectMapper.readValue(split.getSplitEnd(), Object[].class); - finishedSnapshotSplitInfos.add( new FinishedSnapshotSplitInfo( TableId.parse(split.getTableId()), split.getSplitId(), - splitStart, - splitEnd, + split.getSplitStart(), + split.getSplitEnd(), binlogOffset)); } } @@ -529,7 +417,7 @@ private List startSplitChunks( List remainingTables = new ArrayList<>(); if (snapshotTable != null) { // need add database name - String database = config.get(LoadConstants.DATABASE_NAME); + String database = config.get(LoadConstants.DATABASE); remainingTables.add(TableId.parse(database + "." + snapshotTable)); } List remainingSplits = new ArrayList<>(); @@ -552,7 +440,6 @@ private List startSplitChunks( private SplitRecords pollSplitRecordsWithSplit(MySqlSplit split, JobConfig jobConfig) throws Exception { Preconditions.checkState(split != null, "split is null"); - JobRuntimeContext jobContext = getJobRuntimeContext(jobConfig.getJobId()); Iterator dataIt = null; String currentSplitId = null; DebeziumReader currentReader = null; @@ -562,14 +449,14 @@ private SplitRecords pollSplitRecordsWithSplit(MySqlSplit split, JobConfig jobCo } else if (split instanceof MySqlBinlogSplit) { currentReader = getBinlogSplitReader(jobConfig); } - jobContext.setCurrentReader(currentReader); + jobRuntimeContext.setCurrentReader(currentReader); currentReader.submitSplit(split); currentSplitId = split.splitId(); // make split record available Thread.sleep(100); dataIt = currentReader.pollSplitRecords(); if (currentReader instanceof SnapshotSplitReader) { - closeSnapshotReader(jobConfig.getJobId()); + closeSnapshotReader(); } return dataIt == null ? null : new SplitRecords(currentSplitId, dataIt.next()); } @@ -587,8 +474,7 @@ private SplitRecords pollSplitRecordsWithCurrentReader( private SnapshotSplitReader getSnapshotSplitReader(JobConfig config) { MySqlSourceConfig sourceConfig = getSourceConfig(config); - JobRuntimeContext jobContext = getJobRuntimeContext(config.getJobId()); - SnapshotSplitReader snapshotReader = jobContext.getSnapshotReader(); + SnapshotSplitReader snapshotReader = jobRuntimeContext.getSnapshotReader(); if (snapshotReader == null) { final MySqlConnection jdbcConnection = DebeziumUtils.createMySqlConnection(sourceConfig); @@ -597,15 +483,14 @@ private SnapshotSplitReader getSnapshotSplitReader(JobConfig config) { final StatefulTaskContext statefulTaskContext = new StatefulTaskContext(sourceConfig, binaryLogClient, jdbcConnection); snapshotReader = new SnapshotSplitReader(statefulTaskContext, 0); - jobContext.setSnapshotReader(snapshotReader); + jobRuntimeContext.setSnapshotReader(snapshotReader); } return snapshotReader; } private BinlogSplitReader getBinlogSplitReader(JobConfig config) { MySqlSourceConfig sourceConfig = getSourceConfig(config); - JobRuntimeContext jobContext = getJobRuntimeContext(config.getJobId()); - BinlogSplitReader binlogReader = jobContext.getBinlogReader(); + BinlogSplitReader binlogReader = jobRuntimeContext.getBinlogReader(); if (binlogReader == null) { final MySqlConnection jdbcConnection = DebeziumUtils.createMySqlConnection(sourceConfig); @@ -614,43 +499,37 @@ private BinlogSplitReader getBinlogSplitReader(JobConfig config) { final StatefulTaskContext statefulTaskContext = new StatefulTaskContext(sourceConfig, binaryLogClient, jdbcConnection); binlogReader = new BinlogSplitReader(statefulTaskContext, 0); - jobContext.setBinlogReader(binlogReader); + jobRuntimeContext.setBinlogReader(binlogReader); } return binlogReader; } - private void closeSnapshotReader(Long jobId) { - JobRuntimeContext jobContext = jobContexts.get(jobId); - if (jobContext == null) { - return; - } - SnapshotSplitReader reusedSnapshotReader = jobContext.getSnapshotReader(); + private void closeSnapshotReader() { + SnapshotSplitReader reusedSnapshotReader = jobRuntimeContext.getSnapshotReader(); if (reusedSnapshotReader != null) { LOG.debug( "Close snapshot reader {}", reusedSnapshotReader.getClass().getCanonicalName()); reusedSnapshotReader.close(); - DebeziumReader currentReader = jobContext.getCurrentReader(); + DebeziumReader currentReader = + jobRuntimeContext.getCurrentReader(); if (reusedSnapshotReader == currentReader) { - jobContext.setCurrentReader(null); + jobRuntimeContext.setCurrentReader(null); } - jobContext.setSnapshotReader(null); + jobRuntimeContext.setSnapshotReader(null); } } - private void closeBinlogReader(Long jobId) { - JobRuntimeContext jobContext = jobContexts.get(jobId); - if (jobContext == null) { - return; - } - BinlogSplitReader reusedBinlogReader = jobContext.getBinlogReader(); + private void closeBinlogReader() { + BinlogSplitReader reusedBinlogReader = jobRuntimeContext.getBinlogReader(); if (reusedBinlogReader != null) { LOG.debug("Close binlog reader {}", reusedBinlogReader.getClass().getCanonicalName()); reusedBinlogReader.close(); - DebeziumReader currentReader = jobContext.getCurrentReader(); + DebeziumReader currentReader = + jobRuntimeContext.getCurrentReader(); if (reusedBinlogReader == currentReader) { - jobContext.setCurrentReader(null); + jobRuntimeContext.setCurrentReader(null); } - jobContext.setBinlogReader(null); + jobRuntimeContext.setBinlogReader(null); } } @@ -659,21 +538,57 @@ private MySqlSourceConfig getSourceConfig(JobConfig config) { } @Override - public void close(Long jobId) { - JobRuntimeContext jobContext = jobContexts.remove(jobId); - if (jobContext == null) { - return; + public Map extractSnapshotOffset(Object splitState, Object split) { + if (splitState == null) { + return null; + } + MySqlSplitState mysqlSplitState = (MySqlSplitState) splitState; + MySqlSplit mysqlSplit = (MySqlSplit) split; + BinlogOffset highWatermark = mysqlSplitState.asSnapshotSplitState().getHighWatermark(); + Map offsetRes = new HashMap<>(highWatermark.getOffset()); + if (mysqlSplit != null) { + offsetRes.put(SPLIT_ID, mysqlSplit.splitId()); + } + return offsetRes; + } + + @Override + public Map extractBinlogOffset(Object split, boolean pureBinlogPhase) { + if (split == null) { + return null; + } + MySqlSplit mysqlSplit = (MySqlSplit) split; + Map offsetRes = mysqlSplit.asBinlogSplit().getStartingOffset().getOffset(); + offsetRes.put(SPLIT_ID, BINLOG_SPLIT_ID); + offsetRes.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + return offsetRes; + } + + @Override + public String getSplitId(Object split) { + if (split == null) { + return null; } - jobContext.close(); + MySqlSplit mysqlSplit = (MySqlSplit) split; + return mysqlSplit.splitId(); + } + + @Override + public void close(Long jobId) { + jobRuntimeContext.close(); LOG.info("Close source reader for job {}", jobId); } + @Override + public void finishSplitRecords() { + jobRuntimeContext.setCurrentSplitRecords(null); + } + private Map getTableSchemas(JobConfig config) { - JobRuntimeContext jobContext = getJobRuntimeContext(config.getJobId()); - Map schemas = jobContext.getTableSchemas(); + Map schemas = jobRuntimeContext.getTableSchemas(); if (schemas == null) { schemas = discoverTableSchemas(config); - jobContext.setTableSchemas(schemas); + jobRuntimeContext.setTableSchemas(schemas); } return schemas; } @@ -690,63 +605,13 @@ private Map discoverTableSchemas(JobConfig co } } - private JobRuntimeContext getJobRuntimeContext(Long jobId) { - Objects.requireNonNull(jobId, "jobId"); - return jobContexts.computeIfAbsent(jobId, JobRuntimeContext::new); - } - - /** 临时保存一次 read 调用中与 split 相关的状态,避免大量方法参数。 */ - private static final class SplitProcessingContext { - private SplitRecords splitRecords; - private MySqlSplit split; - private boolean readBinlog = true; - private boolean pureBinlogPhase = true; - - private SplitRecords getSplitRecords() { - return splitRecords; - } - - private void setSplitRecords(SplitRecords splitRecords) { - this.splitRecords = splitRecords; - } - - private MySqlSplit getSplit() { - return split; - } - - private void setSplit(MySqlSplit split) { - this.split = split; - } - - private boolean isReadBinlog() { - return readBinlog; - } - - private void setReadBinlog(boolean readBinlog) { - this.readBinlog = readBinlog; - } - - private boolean isPureBinlogPhase() { - return pureBinlogPhase; - } - - private void setPureBinlogPhase(boolean pureBinlogPhase) { - this.pureBinlogPhase = pureBinlogPhase; - } - } - private static final class JobRuntimeContext { - private final long jobId; private SnapshotSplitReader snapshotReader; private BinlogSplitReader binlogReader; private DebeziumReader currentReader; private Map tableSchemas; private SplitRecords currentSplitRecords; - private JobRuntimeContext(Long jobId) { - this.jobId = jobId; - } - private SnapshotSplitReader getSnapshotReader() { return snapshotReader; } @@ -803,36 +668,21 @@ private void close() { } /** - * Filtered record iterator that only returns data change records, filtering out watermark, heartbeat and other events. - * This is a private static inner class that encapsulates record filtering logic, making the main method cleaner. + * Filtered record iterator that only returns data change records, filtering out watermark, + * heartbeat and other events. This is a private static inner class that encapsulates record + * filtering logic, making the main method cleaner. */ private static class FilteredRecordIterator implements Iterator { private final Iterator sourceIterator; private final MySqlSplitState splitState; - private final int fetchSize; - private final boolean readBinlog; - private final boolean pureBinlogPhase; - private final SplitReadResult result; - private SourceRecord nextRecord; - private int count = 0; - - FilteredRecordIterator( - SplitRecords currentSplitRecords, - MySqlSplitState splitState, - int fetchSize, - boolean readBinlog, - boolean pureBinlogPhase, - SplitReadResult result) { + + FilteredRecordIterator(SplitRecords currentSplitRecords, MySqlSplitState splitState) { this.sourceIterator = currentSplitRecords != null && !currentSplitRecords.isEmpty() ? currentSplitRecords.getIterator() : null; this.splitState = splitState; - this.fetchSize = fetchSize; - this.readBinlog = readBinlog; - this.pureBinlogPhase = pureBinlogPhase; - this.result = result; } @Override @@ -843,9 +693,6 @@ public boolean hasNext() { if (nextRecord != null) { return true; } - if (count >= fetchSize) { - return false; - } while (sourceIterator.hasNext()) { SourceRecord element = sourceIterator.next(); @@ -858,16 +705,6 @@ public boolean hasNext() { LOG.debug("Receive heartbeat event: {}", element); } else if (RecordUtils.isDataChangeRecord(element)) { nextRecord = element; - count++; - - // update meta - Map lastMeta = - RecordUtils.getBinlogPosition(element).getOffset(); - if (readBinlog) { - lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); - lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); - result.setLastMeta(lastMeta); - } return true; } else { LOG.debug("Ignore event: {}", element); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java index cd8b85872d1a66..5ed92d4c56fe97 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java @@ -1,20 +1,32 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.cdcclient.source.split; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; + +@Getter +@Setter +@AllArgsConstructor +@NoArgsConstructor public abstract class AbstractSourceSplit implements SourceSplit { private static final long serialVersionUID = 1L; protected String splitId; - - public AbstractSourceSplit() {} - - public AbstractSourceSplit(String splitId) { - this.splitId = splitId; - } - - public String getSplitId() { - return splitId; - } - - public void setSplitId(String splitId) { - this.splitId = splitId; - } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java index a7466feb199e46..8d31ec46d53b6e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java @@ -1,28 +1,36 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.cdcclient.source.split; import java.util.Map; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.Setter; +@Getter +@Setter +@EqualsAndHashCode(callSuper = true) public class BinlogSplit extends AbstractSourceSplit { private static final long serialVersionUID = 1L; private Map offset; - public BinlogSplit() {} - public BinlogSplit(String splitId, Map offset) { super(splitId); this.offset = offset; } - - public Map getOffset() { - return offset; - } - - public void setOffset(Map offset) { - this.offset = offset; - } - - @Override - public String toString() { - return "BinlogSplit{" + "offset=" + offset + ", splitId='" + splitId + '\'' + '}'; - } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java index 92fe48fc73f607..1d63f30d6f6e76 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java @@ -1,25 +1,53 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.cdcclient.source.split; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.List; import java.util.Map; +import java.util.Optional; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; +import org.apache.flink.cdc.common.utils.Preconditions; +@Getter +@Setter +@EqualsAndHashCode(callSuper = true) +@NoArgsConstructor public class SnapshotSplit extends AbstractSourceSplit { private static final long serialVersionUID = 1L; + private static ObjectMapper objectMapper = new ObjectMapper(); private String tableId; - private String splitKey; - private String splitStart; - private String splitEnd; + private List splitKey; + private Object[] splitStart; + private Object[] splitEnd; private Map highWatermark; - public SnapshotSplit() { - super(); - } - public SnapshotSplit( String splitId, String tableId, - String splitKey, - String splitStart, - String splitEnd, + List splitKey, + Object[] splitStart, + Object[] splitEnd, Map highWatermark) { super(splitId); this.tableId = tableId; @@ -29,66 +57,47 @@ public SnapshotSplit( this.highWatermark = highWatermark; } - public String getTableId() { - return tableId; - } - - public void setTableId(String tableId) { - this.tableId = tableId; - } - - public String getSplitKey() { - return splitKey; - } - - public void setSplitKey(String splitKey) { - this.splitKey = splitKey; - } + public static SnapshotSplit fromMap(Map map) throws JsonProcessingException { + if (map == null || map.isEmpty()) { + return null; + } - public String getSplitStart() { - return splitStart; - } + SnapshotSplit split = new SnapshotSplit(); + String splitId = map.get("splitId"); + String tableId = map.get("tableId"); + String splitKeyStr = map.get("splitKey"); + Preconditions.checkNotNull(splitKeyStr, "splitKey must not be null"); + List splitKey = + objectMapper.readValue(splitKeyStr, new TypeReference>() {}); - public void setSplitStart(String splitStart) { - this.splitStart = splitStart; - } + split.setSplitId(splitId); + split.setTableId(tableId); + split.setSplitKey(splitKey); - public String getSplitEnd() { - return splitEnd; - } + String splitStartStr = map.get("splitStart"); + if (splitStartStr != null) { + Object[] splitStart = objectMapper.readValue(splitStartStr, Object[].class); + split.setSplitStart(splitStart); + } - public void setSplitEnd(String splitEnd) { - this.splitEnd = splitEnd; - } + String splitEndStr = map.get("splitEnd"); + if (splitEndStr != null) { + Object[] splitEnd = objectMapper.readValue(splitEndStr, Object[].class); + split.setSplitEnd(splitEnd); + } - public Map getHighWatermark() { - return highWatermark; - } + String highWatermarkStr = map.get("highWatermark"); + if (highWatermarkStr != null) { + Map highWatermark = + objectMapper.readValue( + highWatermarkStr, new TypeReference>() {}); + split.setHighWatermark(highWatermark); + } - public void setHighWatermark(Map highWatermark) { - this.highWatermark = highWatermark; + return split; } - @Override - public String toString() { - return "SnapshotSplit{" - + "tableId='" - + tableId - + '\'' - + ", splitKey='" - + splitKey - + '\'' - + ", splitStart='" - + splitStart - + '\'' - + ", splitEnd='" - + splitEnd - + '\'' - + ", highWatermark=" - + highWatermark - + ", splitId='" - + splitId - + '\'' - + '}'; + public static String getOrEmptyArray(Map map, String key) { + return Optional.ofNullable(map.get(key)).orElse("[]"); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SourceSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SourceSplit.java index 70c8be085ea28a..274927d43e4c61 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SourceSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SourceSplit.java @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.cdcclient.source.split; public interface SourceSplit {} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index 97555ac5844912..9d3e1acf5d9205 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -17,10 +17,11 @@ package org.apache.doris.cdcclient.utils; +import java.util.Map; +import java.util.Properties; +import org.apache.commons.lang3.StringUtils; import org.apache.doris.cdcclient.constants.LoadConstants; import org.apache.doris.cdcclient.model.JobConfig; - -import org.apache.commons.lang3.StringUtils; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; @@ -28,9 +29,6 @@ import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetBuilder; import org.apache.flink.cdc.connectors.mysql.table.StartupOptions; -import java.util.Map; -import java.util.Properties; - public class ConfigUtil { public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { @@ -41,15 +39,15 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { configFactory.username(cdcConfig.get(LoadConstants.USERNAME)); configFactory.password(cdcConfig.get(LoadConstants.PASSWORD)); - String databaseName = cdcConfig.get(LoadConstants.DATABASE_NAME); + String databaseName = cdcConfig.get(LoadConstants.DATABASE); configFactory.databaseList(databaseName); configFactory.serverId(String.valueOf(Math.abs(config.getJobId().hashCode()))); configFactory.includeSchemaChanges(false); - String includingTables = cdcConfig.getOrDefault(LoadConstants.INCLUDE_TABLES_LIST, ".*"); + String includingTables = cdcConfig.getOrDefault(LoadConstants.INCLUDE_TABLES, ".*"); String includingPattern = String.format("(%s)\\.(%s)", databaseName, includingTables); - String excludingTables = cdcConfig.get(LoadConstants.EXCLUDE_TABLES_LIST); + String excludingTables = cdcConfig.get(LoadConstants.EXCLUDE_TABLES); if (StringUtils.isEmpty(excludingTables)) { configFactory.tableList(includingPattern); } else { @@ -113,8 +111,10 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { jdbcProperteis.put("useSSL", "false"); configFactory.jdbcProperties(jdbcProperteis); - // for debug - // configFactory.splitSize(1); + if (cdcConfig.containsKey(LoadConstants.SPLIT_SIZE)) { + configFactory.splitSize(Integer.parseInt(cdcConfig.get(LoadConstants.SPLIT_SIZE))); + } + return configFactory.createConfig(0); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/HttpUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/HttpUtil.java index 7ac3d5dc2cf40a..4d1356003fba60 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/HttpUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/HttpUtil.java @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.cdcclient.utils; import org.apache.http.client.config.RequestConfig; From 03fa875eca4c62e780d127384ae7b77aa0fb5711 Mon Sep 17 00:00:00 2001 From: wudi Date: Fri, 28 Nov 2025 12:04:39 +0800 Subject: [PATCH 04/27] add be request cdc client --- be/src/common/config.cpp | 4 + be/src/common/config.h | 6 + be/src/runtime/cdc_client_manager.cpp | 395 ++++++++++++++++++++++++++ be/src/runtime/cdc_client_manager.h | 68 +++++ be/src/runtime/exec_env.h | 3 + be/src/runtime/exec_env_init.cpp | 3 + be/src/service/internal_service.cpp | 28 ++ be/src/service/internal_service.h | 10 + gensrc/proto/internal_service.proto | 13 + 9 files changed, 530 insertions(+) create mode 100644 be/src/runtime/cdc_client_manager.cpp create mode 100644 be/src/runtime/cdc_client_manager.h diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 29c452e89be705..21d8f17ce3b8d8 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -65,6 +65,8 @@ DEFINE_Int32(brpc_port, "8060"); DEFINE_Int32(arrow_flight_sql_port, "8050"); +DEFINE_Int32(cdc_client_port, "9096"); + // If the external client cannot directly access priority_networks, set public_host to be accessible // to external client. // There are usually two usage scenarios: @@ -753,6 +755,8 @@ DEFINE_mInt32(max_consumer_num_per_group, "3"); // this should be larger than FE config 'max_routine_load_task_num_per_be' (default 5) DEFINE_Int32(max_routine_load_thread_pool_size, "1024"); +DEFINE_mInt32(max_cdc_client_thread_pool_size, "128"); + // the timeout of condition variable wait in blocking_get and blocking_put DEFINE_mInt32(blocking_queue_cv_wait_timeout_ms, "1000"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 4733b06be5f205..490264e9b7ef74 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -100,6 +100,9 @@ DECLARE_Int32(brpc_port); // Default -1, do not start arrow flight sql server. DECLARE_Int32(arrow_flight_sql_port); +// port for cdc client scan oltp cdc data +DECLARE_Int32(cdc_client_port); + // If the external client cannot directly access priority_networks, set public_host to be accessible // to external client. // There are usually two usage scenarios: @@ -778,6 +781,9 @@ DECLARE_mInt32(max_consumer_num_per_group); // this should be larger than FE config 'max_routine_load_task_num_per_be' (default 5) DECLARE_Int32(max_routine_load_thread_pool_size); +// Thread pool size for CDC client manager +DECLARE_mInt32(max_cdc_client_thread_pool_size); + // max external scan cache batch count, means cache max_memory_cache_batch_count * batch_size row // default is 20, batch_size's default value is 1024 means 20 * 1024 rows will be cached DECLARE_mInt32(max_memory_sink_batch_count); diff --git a/be/src/runtime/cdc_client_manager.cpp b/be/src/runtime/cdc_client_manager.cpp new file mode 100644 index 00000000000000..4ad559ab0cef3e --- /dev/null +++ b/be/src/runtime/cdc_client_manager.cpp @@ -0,0 +1,395 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "runtime/cdc_client_manager.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#ifndef __APPLE__ +#include +#endif + +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/status.h" +#include "http/http_client.h" +#include "runtime/cluster_info.h" +#include "runtime/exec_env.h" +#include "service/backend_options.h" +#include "util/thrift_util.h" +#include "util/threadpool.h" + +namespace doris { + +namespace { +// Handle SIGCHLD signal to prevent zombie processes +void handle_sigchld(int sig_no) { + int status = 0; + pid_t pid = waitpid(0, &status, WNOHANG); + LOG(INFO) << "handle cdc process exit, result: " << pid << ", status: " << status; +} + +// Check CDC client health +Status check_cdc_client_health(int retry_times, int sleep_time, std::string& health_response) { + const std::string cdc_health_url = "http://" + BackendOptions::get_localhost() + ":" + + std::to_string(doris::config::cdc_client_port) + + "/actuator/health"; + + auto health_request = [cdc_health_url, &health_response](HttpClient* client) { + RETURN_IF_ERROR(client->init(cdc_health_url)); + client->set_timeout_ms(5000); + RETURN_IF_ERROR(client->execute(&health_response)); + return Status::OK(); + }; + + Status status = HttpClient::execute_with_retry(retry_times, sleep_time, health_request); + + if (!status.ok()) { + return Status::InternalError( + fmt::format("CDC client health check failed: url={}", cdc_health_url)); + } + + bool is_up = health_response.find("UP") != std::string::npos; + + if (!is_up) { + return Status::InternalError(fmt::format("CDC client unhealthy: url={}, response={}", + cdc_health_url, health_response)); + } + + return Status::OK(); +} + +// Start CDC client process +Status start_cdc_client(const std::string& params, PForwardCdcClientResult* result) { + Status st = Status::OK(); + + // Check DORIS_HOME environment variable + const char* doris_home = getenv("DORIS_HOME"); + if (!doris_home) { + st = Status::InternalError("DORIS_HOME environment variable is not set"); + if (result) { + st.to_protobuf(result->mutable_status()); + } + return st; + } + + // Check LOG_DIR environment variable + const char* log_dir = getenv("LOG_DIR"); + if (!log_dir) { + st = Status::InternalError("LOG_DIR environment variable is not set"); + if (result) { + st.to_protobuf(result->mutable_status()); + } + return st; + } + + string cdc_jar_path = string(doris_home) + "/lib/cdc-client/cdc-client.jar"; + string cdc_jar_port = "--server.port=" + std::to_string(doris::config::cdc_client_port); + string backend_host_port = + BackendOptions::get_localhost() + ":" + std::to_string(config::webserver_port); + string cdc_jar_params = params; + string java_opts = "-Dlog.path=" + string(log_dir); + + // check cdc jar exists + struct stat buffer; + if (stat(cdc_jar_path.c_str(), &buffer) != 0) { + st = Status::InternalError("Can not find cdc-client.jar."); + if (result) { + st.to_protobuf(result->mutable_status()); + } + return st; + } + + // check cdc process already started + string check_response; + auto check_st = check_cdc_client_health(1, 0, check_response); + if (check_st.ok()) { + LOG(INFO) << "cdc client already started."; + return Status::OK(); + } else { + LOG(INFO) << "cdc client not started, to start."; + } + + const auto* java_home = getenv("JAVA_HOME"); + if (!java_home) { + st = Status::InternalError("Can not find java home."); + if (result) { + st.to_protobuf(result->mutable_status()); + } + return st; + } + std::string path(java_home); + + // Capture signal to prevent child process from becoming a zombie process + struct sigaction act; + act.sa_flags = 0; + act.sa_handler = handle_sigchld; + sigaction(SIGCHLD, &act, NULL); + LOG(INFO) << "Start to fork cdc client process with " << path; + + // If has a forked process, the child process fails to start and will automatically exit + pid_t pid = ::fork(); + if (pid < 0) { + // Fork failed + st = Status::InternalError("Fork cdc client failed."); + if (result) { + st.to_protobuf(result->mutable_status()); + } + return st; + } else if (pid == 0) { + // When the parent process is killed, the child process also needs to exit +#ifndef __APPLE__ + prctl(PR_SET_PDEATHSIG, SIGKILL); +#endif + + LOG(INFO) << "Cdc client child process ready to start, " << pid << ", response=" + << std::endl; + std::cout << "Cdc client child process ready to start." << std::endl; + std::string java_bin = path + "/bin/java"; + execlp(java_bin.c_str(), "java", java_opts.c_str(), "-jar", cdc_jar_path.c_str(), + cdc_jar_port.c_str(), backend_host_port.c_str(), cdc_jar_params.c_str(), + (char*)NULL); + std::cerr << "Cdc client child process error." << std::endl; + exit(1); + } else { + // Waiting for cdc to start, failed after more than 30 seconds + string health_response; + Status status = check_cdc_client_health(5, 6, health_response); + if (!status.ok()) { + LOG(ERROR) << "Failed to start cdc client process, status=" << status.to_string() + << ", response=" << health_response; + st = Status::InternalError("Start cdc client failed."); + if (result) { + st.to_protobuf(result->mutable_status()); + } + } else { + LOG(INFO) << "Start cdc client success, status=" << status.to_string() + << ", response=" << health_response; + } + } + return st; +} + +} // anonymous namespace + +CdcClientManager::CdcClientManager(ExecEnv* exec_env) : _exec_env(exec_env) { + static_cast(ThreadPoolBuilder("CdcClientThreadPool") + .set_min_threads(1) + .set_max_threads(config::max_cdc_client_thread_pool_size) + .build(&_thread_pool)); +} + +CdcClientManager::~CdcClientManager() { + stop(); + LOG(INFO) << "CdcClientManager is destroyed"; +} + +void CdcClientManager::stop() { + if (_thread_pool) { + _thread_pool->shutdown(); + } + LOG(INFO) << "CdcClientManager is stopped"; +} + +void CdcClientManager::request_cdc_client_impl(const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) { + VLOG_RPC << "request to cdc client, api " << request->api(); + brpc::ClosureGuard closure_guard(done); + + // Start CDC client if not started + Status start_st = start_cdc_client(request->params(), nullptr); + if (!start_st.ok()) { + LOG(ERROR) << "Failed to start CDC client, status=" << start_st.to_string(); + start_st.to_protobuf(result->mutable_status()); + return; + } + + // Send HTTP request synchronously (this is called from heavy_work_pool, so it's already async) + std::string cdc_response; + Status st = send_request_to_cdc_client(request->api(), request->params(), &cdc_response); + result->set_response(cdc_response); + st.to_protobuf(result->mutable_status()); +} + + +Status CdcClientManager::send_request_to_cdc_client(const std::string& api, + const std::string& params_body, + std::string* response) { + std::string remote_url_prefix = fmt::format("http://{}:{}{}", + BackendOptions::get_localhost(), + doris::config::cdc_client_port, + api); + + auto cdc_request = [&remote_url_prefix, response, ¶ms_body](HttpClient* client) { + RETURN_IF_ERROR(client->init(remote_url_prefix)); + client->set_timeout_ms(60 * 1000); + if (!params_body.empty()) { + client->set_payload(params_body); + } + client->set_content_type("application/json"); + client->set_method(POST); + RETURN_IF_ERROR(client->execute(response)); + return Status::OK(); + }; + + return HttpClient::execute_with_retry(3, 1, cdc_request); +} + +Status CdcClientManager::extract_meta_from_response(const std::string& cdc_response, + std::string* meta_json) { + rapidjson::Document doc; + if (doc.Parse(cdc_response.c_str()).HasParseError()) { + return Status::InternalError("Failed to parse CDC response JSON"); + } + + // Check if there is a data field + if (!doc.HasMember("data") || !doc["data"].IsObject()) { + return Status::InternalError("CDC response missing 'data' field or not an object"); + } + + // Check if there is a meta field + const rapidjson::Value& data = doc["data"]; + if (!data.HasMember("meta") || !data["meta"].IsObject()) { + return Status::InternalError("CDC response missing 'meta' field or not an object"); + } + + // Extract meta object and serialize as JSON string + const rapidjson::Value& meta = data["meta"]; + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + meta.Accept(writer); + *meta_json = buffer.GetString(); + return Status::OK(); +} + +Status CdcClientManager::commit_transaction(const std::string& txn_id, + const std::string& meta_json) { + TLoadTxnCommitRequest commit_request; + commit_request.__set_txn_id(txn_id); + + StreamingTaskCommitAttachmentPB attachment; + attachment.set_offset(meta_json); + + commit_request.__set_txnCommitAttachment(attachment); + + TNetworkAddress master_addr = _exec_env->cluster_info()->master_fe_addr; + TLoadTxnCommitResult commit_result; + + Status rpc_st = ThriftRpcHelper::rpc( + master_addr.hostname, master_addr.port, + [&commit_request, &commit_result](FrontendServiceConnection& client) { + client->loadTxnCommit(commit_result, commit_request); + }, + config::txn_commit_rpc_timeout_ms); + + if (!rpc_st.ok()) { + return Status::InternalError( + fmt::format("Failed to call FE loadTxnCommit, rpc_status={}, txn_id={}", + rpc_st.to_string(), txn_id)); + } + + Status result_status = Status::create(commit_result.status); + if (!result_status.ok()) { + return Status::InternalError( + fmt::format("FE loadTxnCommit returned error, status={}, txn_id={}", + result_status.to_string(), txn_id)); + } + + return Status::OK(); +} + +void CdcClientManager::execute_cdc_scan_commit_impl(const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) { + VLOG_RPC << "forward request to cdc client, api " << request->api(); + brpc::ClosureGuard closure_guard(done); + + // Start CDC client if not started + Status start_st = start_cdc_client(request->params(), nullptr); + if (!start_st.ok()) { + LOG(ERROR) << "Failed to start CDC client, status=" << start_st.to_string(); + start_st.to_protobuf(result->mutable_status()); + return; + } + + // Extract parameters from request + std::string api = request->api(); + std::string txn_id = request->txn_id(); + std::string params_body = request->params(); + + // Submit async task to handle CDC scan and commit using internal thread_pool + Status submit_st = _thread_pool->submit_func([this, api, params_body, txn_id]() { + // Request cdc client to read and load data + std::string cdc_response; + Status st = send_http_request_to_cdc_client(api, params_body, &cdc_response); + if (!st.ok()) { + LOG(ERROR) << "CDC client HTTP request failed, status=" << st.to_string() + << ", api=" << api << ", txn_id=" << txn_id; + return; + } + + LOG(INFO) << "CDC client HTTP request success, response=" << cdc_response + << ", txn_id=" << txn_id; + + // Parse JSON, extract data.meta part + std::string meta_json; + Status parse_st = extract_meta_from_response(cdc_response, &meta_json); + if (!parse_st.ok()) { + LOG(ERROR) << "Failed to extract meta from CDC response, txn_id=" << txn_id + << ", status=" << parse_st.to_string(); + return; + } + + // Commit txn + Status commit_st = commit_transaction(txn_id, meta_json); + if (!commit_st.ok()) { + LOG(ERROR) << "Failed to commit CDC transaction, txn_id=" << txn_id + << ", status=" << commit_st.to_string(); + return; + } + + LOG(INFO) << "Successfully committed CDC transaction to FE, txn_id=" << txn_id; + }); + + if (!submit_st.ok()) { + LOG(ERROR) << "Failed to submit CDC client async task to thread pool, " + << "status=" << submit_st.to_string() << ", txn_id=" << txn_id; + submit_st.to_protobuf(result->mutable_status()); + return; + } + + // Return success to FE immediately after task is successfully submitted + Status::OK().to_protobuf(result->mutable_status()); +} + +} // namespace doris + diff --git a/be/src/runtime/cdc_client_manager.h b/be/src/runtime/cdc_client_manager.h new file mode 100644 index 00000000000000..5debdaa85a75ce --- /dev/null +++ b/be/src/runtime/cdc_client_manager.h @@ -0,0 +1,68 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include + +#include "common/status.h" +#include "util/threadpool.h" + +namespace google::protobuf { +class Closure; +class RpcController; +} // namespace google::protobuf + +namespace doris { + +class ExecEnv; + +class CdcClientManager { +public: + explicit CdcClientManager(ExecEnv* exec_env); + ~CdcClientManager(); + + void stop(); + + // Request CDC client to handle a request + void request_cdc_client_impl(const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done); + + // Execute CDC scan and commit transaction + void execute_cdc_scan_commit_impl(const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done); + +private: + Status send_request_to_cdc_client(const std::string& api, + const std::string& params_body, + std::string* response); + + Status extract_meta_from_response(const std::string& cdc_response, + std::string* meta_json); + + Status commit_transaction(const std::string& txn_id, const std::string& meta_json); + + ExecEnv* _exec_env = nullptr; + std::unique_ptr _thread_pool; +}; + +} // namespace doris + diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 1258f945fa2ef5..50c27341b1ab0a 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -118,6 +118,7 @@ class HeartbeatFlags; class FrontendServiceClient; class FileMetaCache; class GroupCommitMgr; +class CdcClientManager; class TabletSchemaCache; class TabletColumnObjectPool; class UserFunctionCache; @@ -277,6 +278,7 @@ class ExecEnv { SmallFileMgr* small_file_mgr() { return _small_file_mgr; } doris::vectorized::SpillStreamManager* spill_stream_mgr() { return _spill_stream_mgr; } GroupCommitMgr* group_commit_mgr() { return _group_commit_mgr; } + CdcClientManager* cdc_client_mgr() { return _cdc_client_mgr; } const std::vector& store_paths() const { return _store_paths; } @@ -510,6 +512,7 @@ class ExecEnv { // ip:brpc_port -> frontend_indo std::map _frontends; GroupCommitMgr* _group_commit_mgr = nullptr; + CdcClientManager* _cdc_client_mgr = nullptr; // Maybe we should use unique_ptr, but it need complete type, which means we need // to include many headers, and for some cpp file that do not need class like TabletSchemaCache, diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 0d7ecb7aff4300..26926354d24e9f 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -75,6 +75,7 @@ #include "runtime/external_scan_context_mgr.h" #include "runtime/fragment_mgr.h" #include "runtime/group_commit_mgr.h" +#include "runtime/cdc_client_manager.h" #include "runtime/heartbeat_flags.h" #include "runtime/index_policy/index_policy_mgr.h" #include "runtime/load_channel_mgr.h" @@ -335,6 +336,7 @@ Status ExecEnv::_init(const std::vector& store_paths, RETURN_IF_ERROR(_routine_load_task_executor->init(MemInfo::mem_limit())); _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); _group_commit_mgr = new GroupCommitMgr(this); + _cdc_client_mgr = new CdcClientManager(this); _memtable_memory_limiter = std::make_unique(); _load_stream_map_pool = std::make_unique(); _delta_writer_v2_pool = std::make_unique(); @@ -842,6 +844,7 @@ void ExecEnv::destroy() { SAFE_DELETE(_result_mgr); SAFE_DELETE(_file_meta_cache); SAFE_DELETE(_group_commit_mgr); + SAFE_DELETE(_cdc_client_mgr); SAFE_DELETE(_routine_load_task_executor); SAFE_DELETE(_stream_load_recorder_manager); // _stream_load_executor diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 426afc89fd2b55..bfdf6765b1ef8d 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -2396,5 +2396,33 @@ void PInternalService::get_tablet_rowsets(google::protobuf::RpcController* contr Status::OK().to_protobuf(response->mutable_status()); } +void PInternalService::request_cdc_client(google::protobuf::RpcController* controller, + const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) { + bool ret = _heavy_work_pool.try_offer([this, request, result, done]() { + _exec_env->cdc_client_mgr()->request_cdc_client_impl(request, result, done); + }); + + if (!ret) { + offer_failed(result, done, _heavy_work_pool); + return; + } +} + +void PInternalService::execute_cdc_scan_commit(google::protobuf::RpcController* controller, + const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) { + bool ret = _heavy_work_pool.try_offer([this, request, result, done]() { + _exec_env->cdc_client_mgr()->execute_cdc_scan_commit_impl(request, result, done); + }); + + if (!ret) { + offer_failed(result, done, _heavy_work_pool); + return; + } +} + #include "common/compile_check_avoid_end.h" } // namespace doris diff --git a/be/src/service/internal_service.h b/be/src/service/internal_service.h index d73501bfc80861..ba7a39be4a3722 100644 --- a/be/src/service/internal_service.h +++ b/be/src/service/internal_service.h @@ -232,6 +232,16 @@ class PInternalService : public PBackendService { PGetTabletRowsetsResponse* response, google::protobuf::Closure* done) override; + void request_cdc_client(google::protobuf::RpcController* controller, + const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) override; + + void execute_cdc_scan_commit(google::protobuf::RpcController* controller, + const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) + private: void _exec_plan_fragment_in_pthread(google::protobuf::RpcController* controller, const PExecPlanFragmentRequest* request, diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index 69659aed523e17..1728f88dc1ea3d 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -1117,6 +1117,17 @@ message PFetchPeerDataResponse { repeated CacheBlockPB datas = 2; } +message PRequestCdcClientRequest { + optional string api = 1; + optional string params = 2; + optional int64 txn_id = 3; +} + +message PRequestCdcClientResult { + optional PStatus status = 1; + optional string response = 2; +} + service PBackendService { // If #fragments of a query is < 3, use exec_plan_fragment directly. // If #fragments of a query is >=3, use exec_plan_fragment_prepare + exec_plan_fragment_start @@ -1174,5 +1185,7 @@ service PBackendService { rpc abort_refresh_dictionary(PAbortRefreshDictionaryRequest) returns (PAbortRefreshDictionaryResponse); rpc get_tablet_rowsets(PGetTabletRowsetsRequest) returns (PGetTabletRowsetsResponse); rpc fetch_peer_data(PFetchPeerDataRequest) returns (PFetchPeerDataResponse); + rpc request_cdc_client(PRequestCdcClientRequest) returns (PRequestCdcClientResult); + rpc execute_cdc_scan_commit(PRequestCdcClientRequest) returns (PRequestCdcClientResult); }; From aa857421d08bc5339d944b0ea3a52b32482f275b Mon Sep 17 00:00:00 2001 From: wudi Date: Fri, 28 Nov 2025 17:46:05 +0800 Subject: [PATCH 05/27] add fe rpc interface and test --- .../insert/streaming/StreamingInsertJob.java | 79 +++++++++++++++++++ .../doris/rpc/BackendServiceClient.java | 10 +++ .../apache/doris/rpc/BackendServiceProxy.java | 20 +++++ .../src/main/resources/log4j.properties | 17 +++- 4 files changed, 122 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index dd6bea8084074f..3a8ee70630adac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -60,11 +60,20 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.InternalService.PRequestCdcClientResult; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.resource.Tag; +import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.rpc.RpcException; +import org.apache.doris.system.Backend; +import org.apache.doris.system.BeSelectionPolicy; +import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TCell; +import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TRow; +import org.apache.doris.thrift.TStatusCode; import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnStateChangeCallback; @@ -82,10 +91,15 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -137,9 +151,74 @@ public StreamingInsertJob(String jobName, super(Env.getCurrentEnv().getNextId(), jobName, jobStatus, dbName, comment, createUser, jobConfig, createTimeMs, executeSql); this.properties = properties; + try { + log.info("======fetch cdc split"); + fetchCdcSplit(); + } catch (JobException e) { + throw new RuntimeException(e); + } init(); } + // todo: down to offset provider + private void fetchCdcSplit() throws JobException{ + Backend backend = selectBackend(1L); + Map params = new HashMap<>(); + params.put("jobId", "1"); + params.put("dataSource", "MYSQL"); + params.put("snapshotTable", "user_info"); + Map config = new HashMap<>(); + config.put("host", "127.0.0.1"); + config.put("port", "3308"); + config.put("username", "root"); + config.put("password", "123456"); + config.put("database", "test"); + config.put("include_tables", "user_info"); + params.put("config", config); + + InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() + .setApi("/api/fetchSplits") + .setParams(GsonUtils.GSON.toJson(params)).build(); + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); + InternalService.PRequestCdcClientResult result = null; + try { + Future future = + BackendServiceProxy.getInstance().requestCdcClient(address, request); + result = future.get(); + TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + log.error("Failed to get split from backend, {}", result.getStatus().getErrorMsgs(0)); + throw new JobException("Failed to get split from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + result.getResponse()); + } + } catch (ExecutionException | InterruptedException ex) { + log.error("Get splits error: ", ex); + throw new JobException(ex); + } + log.info("========fetch cdc split {}", result.getResponse()); + } + + public static Backend selectBackend(Long jobId) throws JobException { + Backend backend = null; + BeSelectionPolicy policy = null; + + policy = new BeSelectionPolicy.Builder() + .setEnableRoundRobin(true) + .needLoadAvailable().build(); + List backendIds; + backendIds = Env.getCurrentSystemInfo().selectBackendIdsByPolicy(policy, 1); + if (backendIds.isEmpty()) { + throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); + } + //jobid % backendSize + long index = backendIds.get(jobId.intValue() % backendIds.size()); + backend = Env.getCurrentSystemInfo().getBackend(index); + if (backend == null) { + throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); + } + return backend; + } + + private void init() { try { this.jobProperties = new StreamingJobProperties(properties); diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java index 0cd79e1ba5301c..8b245f2e91b922 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java @@ -209,6 +209,16 @@ public Future abortRefreshDicti return stub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS).abortRefreshDictionary(request); } + public Future requestCdcClient( + InternalService.PRequestCdcClientRequest request) { + return stub.requestCdcClient(request); + } + + public Future executeCdcScanCommit( + InternalService.PRequestCdcClientRequest request) { + return stub.executeCdcScanCommit(request); + } + public void shutdown() { ConnectivityState state = channel.getState(false); LOG.warn("shut down backend service client: {}, channel state: {}", address, state); diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java index 7e15a0503935f4..f7ce54e5ee82fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java @@ -587,4 +587,24 @@ public Future abortDictionaryAs } return null; } + + public Future requestCdcClient(TNetworkAddress address, InternalService.PRequestCdcClientRequest request) { + try { + final BackendServiceClient client = getProxy(address); + return client.requestCdcClient(request); + } catch (Throwable e) { + LOG.warn("request cdc client failed, address={}:{}", address.getHostname(), address.getPort(), e); + } + return null; + } + + public Future executeCdcScanCommit(TNetworkAddress address, InternalService.PRequestCdcClientRequest request) { + try { + final BackendServiceClient client = getProxy(address); + return client.executeCdcScanCommit(request); + } catch (Throwable e) { + LOG.warn("execute cdc scan commit failed, address={}:{}", address.getHostname(), address.getPort(), e); + } + return null; + } } diff --git a/fs_brokers/cdc_client/src/main/resources/log4j.properties b/fs_brokers/cdc_client/src/main/resources/log4j.properties index ecb73d3811cf63..f4c8faf2ff6130 100644 --- a/fs_brokers/cdc_client/src/main/resources/log4j.properties +++ b/fs_brokers/cdc_client/src/main/resources/log4j.properties @@ -16,8 +16,17 @@ # limitations under the License. ################################################################################ -log4j.rootLogger=INFO, console +log4j.rootLogger=INFO, STDOUT, FILE -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c [%t] %x - %m%n \ No newline at end of file +log.path=./logs + +log4j.appender.STDOUT=org.apache.log4j.ConsoleAppender +log4j.appender.STDOUT.layout=org.apache.log4j.PatternLayout +log4j.appender.STDOUT.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.SSS} [%t] %-5p %c - %m%n + +log4j.appender.FILE=org.apache.log4j.RollingFileAppender +log4j.appender.FILE.File=${log.path}/cdc-client.log +log4j.appender.FILE.MaxFileSize=50MB +log4j.appender.FILE.MaxBackupIndex=10 +log4j.appender.FILE.layout=org.apache.log4j.PatternLayout +log4j.appender.FILE.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.SSS} [%t] %-5p %c - %m%n \ No newline at end of file From c12ca09acf4ccf110c8af59be789ee9280f8214f Mon Sep 17 00:00:00 2001 From: JNSimba <676366545@qq.com> Date: Fri, 28 Nov 2025 17:48:24 +0800 Subject: [PATCH 06/27] fix cdc client manager compile --- be/src/runtime/cdc_client_manager.cpp | 37 ++++++++++--------- be/src/runtime/cdc_client_manager.h | 2 +- be/src/service/internal_service.cpp | 1 + be/src/service/internal_service.h | 2 +- .../doris/cdcclient/CdcClientApplication.java | 4 +- .../src/main/resources/log4j.properties | 17 +++++++-- 6 files changed, 36 insertions(+), 27 deletions(-) diff --git a/be/src/runtime/cdc_client_manager.cpp b/be/src/runtime/cdc_client_manager.cpp index 4ad559ab0cef3e..3a0d176c140540 100644 --- a/be/src/runtime/cdc_client_manager.cpp +++ b/be/src/runtime/cdc_client_manager.cpp @@ -46,6 +46,9 @@ #include "service/backend_options.h" #include "util/thrift_util.h" #include "util/threadpool.h" +#include +#include +#include "runtime/client_cache.h" namespace doris { @@ -88,7 +91,7 @@ Status check_cdc_client_health(int retry_times, int sleep_time, std::string& hea } // Start CDC client process -Status start_cdc_client(const std::string& params, PForwardCdcClientResult* result) { +Status start_cdc_client(const std::string& params, PRequestCdcClientResult* result) { Status st = Status::OK(); // Check DORIS_HOME environment variable @@ -111,12 +114,12 @@ Status start_cdc_client(const std::string& params, PForwardCdcClientResult* resu return st; } - string cdc_jar_path = string(doris_home) + "/lib/cdc-client/cdc-client.jar"; - string cdc_jar_port = "--server.port=" + std::to_string(doris::config::cdc_client_port); - string backend_host_port = + const std::string cdc_jar_path = std::string(doris_home) + "/lib/cdc_client/cdc-client.jar"; + const std::string cdc_jar_port = "--server.port=" + std::to_string(doris::config::cdc_client_port); + const std::string backend_host_port = BackendOptions::get_localhost() + ":" + std::to_string(config::webserver_port); - string cdc_jar_params = params; - string java_opts = "-Dlog.path=" + string(log_dir); + const std::string cdc_jar_params = params; + const std::string java_opts = "-Xmx2048m -Dlog.path=" + std::string(log_dir); // check cdc jar exists struct stat buffer; @@ -129,7 +132,7 @@ Status start_cdc_client(const std::string& params, PForwardCdcClientResult* resu } // check cdc process already started - string check_response; + std::string check_response; auto check_st = check_cdc_client_health(1, 0, check_response); if (check_st.ok()) { LOG(INFO) << "cdc client already started."; @@ -181,7 +184,7 @@ Status start_cdc_client(const std::string& params, PForwardCdcClientResult* resu exit(1); } else { // Waiting for cdc to start, failed after more than 30 seconds - string health_response; + std::string health_response; Status status = check_cdc_client_health(5, 6, health_response); if (!status.ok()) { LOG(ERROR) << "Failed to start cdc client process, status=" << status.to_string() @@ -291,15 +294,13 @@ Status CdcClientManager::extract_meta_from_response(const std::string& cdc_respo return Status::OK(); } -Status CdcClientManager::commit_transaction(const std::string& txn_id, +Status CdcClientManager::commit_transaction(int64_t txn_id, const std::string& meta_json) { + TNetworkAddress master_addr = _exec_env->cluster_info()->master_fe_addr; + /** TLoadTxnCommitRequest commit_request; - commit_request.__set_txn_id(txn_id); + commit_request.__set_txnId(txn_id); - StreamingTaskCommitAttachmentPB attachment; - attachment.set_offset(meta_json); - - commit_request.__set_txnCommitAttachment(attachment); TNetworkAddress master_addr = _exec_env->cluster_info()->master_fe_addr; TLoadTxnCommitResult commit_result; @@ -323,14 +324,14 @@ Status CdcClientManager::commit_transaction(const std::string& txn_id, fmt::format("FE loadTxnCommit returned error, status={}, txn_id={}", result_status.to_string(), txn_id)); } - + */ return Status::OK(); } void CdcClientManager::execute_cdc_scan_commit_impl(const PRequestCdcClientRequest* request, PRequestCdcClientResult* result, google::protobuf::Closure* done) { - VLOG_RPC << "forward request to cdc client, api " << request->api(); + VLOG_RPC << "request to cdc client, api " << request->api(); brpc::ClosureGuard closure_guard(done); // Start CDC client if not started @@ -343,14 +344,14 @@ void CdcClientManager::execute_cdc_scan_commit_impl(const PRequestCdcClientReque // Extract parameters from request std::string api = request->api(); - std::string txn_id = request->txn_id(); + int64_t txn_id = request->txn_id(); std::string params_body = request->params(); // Submit async task to handle CDC scan and commit using internal thread_pool Status submit_st = _thread_pool->submit_func([this, api, params_body, txn_id]() { // Request cdc client to read and load data std::string cdc_response; - Status st = send_http_request_to_cdc_client(api, params_body, &cdc_response); + Status st = send_request_to_cdc_client(api, params_body, &cdc_response); if (!st.ok()) { LOG(ERROR) << "CDC client HTTP request failed, status=" << st.to_string() << ", api=" << api << ", txn_id=" << txn_id; diff --git a/be/src/runtime/cdc_client_manager.h b/be/src/runtime/cdc_client_manager.h index 5debdaa85a75ce..bdeb641e1b8382 100644 --- a/be/src/runtime/cdc_client_manager.h +++ b/be/src/runtime/cdc_client_manager.h @@ -58,7 +58,7 @@ class CdcClientManager { Status extract_meta_from_response(const std::string& cdc_response, std::string* meta_json); - Status commit_transaction(const std::string& txn_id, const std::string& meta_json); + Status commit_transaction(const int64_t txn_id, const std::string& meta_json); ExecEnv* _exec_env = nullptr; std::unique_ptr _thread_pool; diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index bfdf6765b1ef8d..e8acddcc4b22b3 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -129,6 +129,7 @@ #include "vec/jsonb/serialize.h" #include "vec/runtime/vdata_stream_mgr.h" #include "vec/sink/vmysql_result_writer.h" +#include "runtime/cdc_client_manager.h" namespace google { namespace protobuf { diff --git a/be/src/service/internal_service.h b/be/src/service/internal_service.h index ba7a39be4a3722..1a9ad77308ac98 100644 --- a/be/src/service/internal_service.h +++ b/be/src/service/internal_service.h @@ -240,7 +240,7 @@ class PInternalService : public PBackendService { void execute_cdc_scan_commit(google::protobuf::RpcController* controller, const PRequestCdcClientRequest* request, PRequestCdcClientResult* result, - google::protobuf::Closure* done) + google::protobuf::Closure* done) override; private: void _exec_plan_fragment_in_pthread(google::protobuf::RpcController* controller, diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java index 94f2a4dc8c716c..cd07d0cf73d291 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java @@ -17,8 +17,8 @@ package org.apache.doris.cdcclient; +import java.util.Arrays; import org.apache.doris.cdcclient.common.Env; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.boot.SpringApplication; @@ -27,8 +27,6 @@ import org.springframework.boot.web.servlet.ServletComponentScan; import org.springframework.boot.web.servlet.support.SpringBootServletInitializer; -import java.util.Arrays; - @SpringBootApplication @EnableConfigurationProperties @ServletComponentScan diff --git a/fs_brokers/cdc_client/src/main/resources/log4j.properties b/fs_brokers/cdc_client/src/main/resources/log4j.properties index ecb73d3811cf63..f4c8faf2ff6130 100644 --- a/fs_brokers/cdc_client/src/main/resources/log4j.properties +++ b/fs_brokers/cdc_client/src/main/resources/log4j.properties @@ -16,8 +16,17 @@ # limitations under the License. ################################################################################ -log4j.rootLogger=INFO, console +log4j.rootLogger=INFO, STDOUT, FILE -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c [%t] %x - %m%n \ No newline at end of file +log.path=./logs + +log4j.appender.STDOUT=org.apache.log4j.ConsoleAppender +log4j.appender.STDOUT.layout=org.apache.log4j.PatternLayout +log4j.appender.STDOUT.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.SSS} [%t] %-5p %c - %m%n + +log4j.appender.FILE=org.apache.log4j.RollingFileAppender +log4j.appender.FILE.File=${log.path}/cdc-client.log +log4j.appender.FILE.MaxFileSize=50MB +log4j.appender.FILE.MaxBackupIndex=10 +log4j.appender.FILE.layout=org.apache.log4j.PatternLayout +log4j.appender.FILE.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.SSS} [%t] %-5p %c - %m%n \ No newline at end of file From 56ac92c2d7f537d76888e4e5fd25494c4bcf5343 Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 2 Dec 2025 19:48:59 +0800 Subject: [PATCH 07/27] add streaming job split chunks and streaming multi task --- .../org/apache/doris/nereids/DorisParser.g4 | 8 +- .../datasource/jdbc/client/JdbcClient.java | 25 +- .../doris/job/common/DataSourceType.java | 5 + .../doris/job/common/LoadConstants.java | 34 ++ .../streaming/AbstractStreamingTask.java | 132 +++++++ .../insert/streaming/StreamingInsertJob.java | 325 ++++++++++++------ .../insert/streaming/StreamingInsertTask.java | 122 +------ .../streaming/StreamingJobSchedulerTask.java | 3 +- .../streaming/StreamingMultiTblTask.java | 61 ++++ .../job/manager/StreamingTaskManager.java | 14 +- .../doris/job/offset/jdbc/JdbcOffset.java | 34 ++ .../offset/jdbc/JdbcSourceOffsetProvider.java | 231 +++++++++++++ .../jdbc/split/AbstractSourceSplit.java | 32 ++ .../job/offset/jdbc/split/BinlogSplit.java | 36 ++ .../job/offset/jdbc/split/SnapshotSplit.java | 103 ++++++ .../job/offset/jdbc/split/SourceSplit.java | 20 ++ .../job/scheduler/StreamingTaskScheduler.java | 13 +- .../doris/job/util/StreamingJobUtils.java | 256 ++++++++++++++ .../nereids/parser/LogicalPlanBuilder.java | 15 +- .../plans/commands/info/CreateJobInfo.java | 76 +++- .../tablefunction/MetadataGenerator.java | 5 +- .../cdcclient/constants/LoadConstants.java | 11 +- .../controller/ClientController.java | 24 +- .../service/PipelineCoordinator.java | 104 +++++- .../doris/cdcclient/utils/ConfigUtil.java | 14 +- 25 files changed, 1423 insertions(+), 280 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/common/DataSourceType.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SourceSplit.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 116264b1f8120f..b313e858792e63 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -99,7 +99,7 @@ materializedViewStatement | SHOW CREATE MATERIALIZED VIEW mvName=multipartIdentifier #showCreateMTMV ; supportedJobStatement - : CREATE JOB label=multipartIdentifier propertyClause? + : CREATE JOB label=multipartIdentifier jobProperties=propertyClause? ON (STREAMING | SCHEDULE( (EVERY timeInterval=INTEGER_VALUE timeUnit=identifier (STARTS (startTime=STRING_LITERAL | CURRENT_TIMESTAMP))? @@ -108,8 +108,10 @@ supportedJobStatement (AT (atTime=STRING_LITERAL | CURRENT_TIMESTAMP)) ) ) - commentSpec? - DO supportedDmlStatement #createScheduledJob + commentSpec? + (FROM sourceType=identifier LEFT_PAREN sourceProperties=propertyItemList RIGHT_PAREN + TO DATABASE targetDb=identifier (LEFT_PAREN targetProperties=propertyItemList RIGHT_PAREN)? + | DO supportedDmlStatement ) #createScheduledJob | PAUSE JOB WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #pauseJob | ALTER JOB (jobName=multipartIdentifier) (propertyClause | supportedDmlStatement | propertyClause supportedDmlStatement) #alterJob | DROP JOB (IF EXISTS)? WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #dropJob diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java index d723371c13e347..53d2a7116bd947 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java @@ -394,16 +394,39 @@ public List getJdbcColumnsInfo(String remoteDbName, String remo public List getColumnsFromJdbc(String remoteDbName, String remoteTableName) { List jdbcTableSchema = getJdbcColumnsInfo(remoteDbName, remoteTableName); + List primaryKeys = getPrimaryKeys(remoteDbName, remoteTableName); List dorisTableSchema = Lists.newArrayListWithCapacity(jdbcTableSchema.size()); for (JdbcFieldSchema field : jdbcTableSchema) { + boolean isKey = primaryKeys.contains(field.getColumnName()); dorisTableSchema.add(new Column(field.getColumnName(), - jdbcTypeToDoris(field), true, null, + jdbcTypeToDoris(field), isKey, null, field.isAllowNull(), field.getRemarks(), true, -1)); } return dorisTableSchema; } + private List getPrimaryKeys(String remoteDbName, String remoteTableName) { + Connection conn = getConnection(); + ResultSet rs = null; + List primaryKeys = Lists.newArrayList(); + try { + DatabaseMetaData databaseMetaData = conn.getMetaData(); + String catalogName = getCatalogName(conn); + rs = databaseMetaData.getPrimaryKeys(catalogName, remoteDbName, remoteTableName); + while (rs.next()) { + String fieldName = rs.getString("COLUMN_NAME"); + primaryKeys.add(fieldName); + } + } catch (SQLException e) { + throw new JdbcClientException("failed to get jdbc primary key info for remote table `%s.%s`: %s", + remoteDbName, remoteTableName, Util.getRootCauseMessage(e)); + } finally { + close(rs, conn); + } + return primaryKeys; + } + // protected methods, for subclass to override protected String getCatalogName(Connection conn) throws SQLException { return conn.getCatalog(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/DataSourceType.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/DataSourceType.java new file mode 100644 index 00000000000000..cf6368aff6a256 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/DataSourceType.java @@ -0,0 +1,5 @@ +package org.apache.doris.job.common; + +public enum DataSourceType { + MYSQL +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java new file mode 100644 index 00000000000000..3148660d31ef77 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java @@ -0,0 +1,34 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.common; + +public class LoadConstants { + public static final String JDBC_URL = "jdbc_url"; + public static final String DRIVER_URL = "driver_url"; + public static final String DRIVER_CLASS = "driver_class"; + public static final String USER = "user"; + public static final String PASSWORD = "password"; + public static final String DATABASE = "database"; + public static final String INCLUDE_TABLES = "include_tables"; + public static final String EXCLUDE_TABLES = "exclude_tables"; + // initial,earliest-offset,latest-offset,specific-offset,timestamp,snapshot + public static final String STARTUP_MODE = "startup_mode"; + public static final String SPLIT_SIZE = "split_size"; + public static final String TABLE_PROPS_PREFIX = "table.create.properties."; + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java new file mode 100644 index 00000000000000..1f8589f897ce63 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java @@ -0,0 +1,132 @@ +package org.apache.doris.job.extensions.insert.streaming; + +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.FeConstants; +import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.job.common.TaskStatus; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.offset.Offset; +import org.apache.doris.load.loadv2.LoadJob; +import org.apache.doris.thrift.TCell; +import org.apache.doris.thrift.TRow; + +import lombok.Getter; +import lombok.Setter; +import lombok.extern.log4j.Log4j2; +import org.apache.commons.lang3.StringUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +@Log4j2 +@Getter +public abstract class AbstractStreamingTask { + private static final int MAX_RETRY = 3; + private int retryCount = 0; + protected String labelName; + protected Offset runningOffset; + protected UserIdentity userIdentity; + @Setter + protected volatile TaskStatus status; + protected String errMsg; + protected long jobId; + protected long taskId; + protected Long createTimeMs; + protected Long startTimeMs; + protected Long finishTimeMs; + @Getter + private AtomicBoolean isCanceled = new AtomicBoolean(false); + + public abstract void before() throws Exception; + public abstract void run() throws JobException ; + public abstract boolean onSuccess() throws JobException; + public abstract void onFail(String errMsg) throws JobException; + public abstract void cancel(boolean needWaitCancelComplete); + public abstract void closeOrReleaseResources(); + + + public void execute() throws JobException { + while (retryCount <= MAX_RETRY) { + try { + before(); + run(); + onSuccess(); + return; + } catch (Exception e) { + if (TaskStatus.CANCELED.equals(status)) { + return; + } + this.errMsg = e.getMessage(); + retryCount++; + if (retryCount > MAX_RETRY) { + log.error("Task execution failed after {} retries.", MAX_RETRY, e); + onFail(e.getMessage()); + return; + } + log.warn("execute streaming task error, job id is {}, task id is {}, retrying {}/{}: {}", + jobId, taskId, retryCount, MAX_RETRY, e.getMessage()); + } finally { + // The cancel logic will call the closeOrReleased Resources method by itself. + // If it is also called here, + // it may result in the inability to obtain relevant information when canceling the task + if (!TaskStatus.CANCELED.equals(status)) { + closeOrReleaseResources(); + } + } + } + } + + /** + * show streaming insert task info detail + */ + public TRow getTvfInfo(String jobName) { + TRow trow = new TRow(); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(this.getTaskId()))); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(this.getJobId()))); + trow.addToColumnValue(new TCell().setStringVal(jobName)); + trow.addToColumnValue(new TCell().setStringVal(this.getLabelName())); + trow.addToColumnValue(new TCell().setStringVal(this.getStatus().name())); + // err msg + trow.addToColumnValue(new TCell().setStringVal(StringUtils.isNotBlank(errMsg) + ? errMsg : FeConstants.null_string)); + + // create time + trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(this.getCreateTimeMs()))); + trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? FeConstants.null_string + : TimeUtils.longToTimeString(this.getStartTimeMs()))); + // load end time + trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(this.getFinishTimeMs()))); + + List loadJobs = Env.getCurrentEnv().getLoadManager() + .queryLoadJobsByJobIds(Arrays.asList(this.getTaskId())); + if (!loadJobs.isEmpty()) { + LoadJob loadJob = loadJobs.get(0); + if (loadJob.getLoadingStatus() != null && loadJob.getLoadingStatus().getTrackingUrl() != null) { + trow.addToColumnValue(new TCell().setStringVal(loadJob.getLoadingStatus().getTrackingUrl())); + } else { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } + + if (loadJob.getLoadStatistic() != null) { + trow.addToColumnValue(new TCell().setStringVal(loadJob.getLoadStatistic().toJson())); + } else { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } + } else { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } + + if (this.getUserIdentity() == null) { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } else { + trow.addToColumnValue(new TCell().setStringVal(this.getUserIdentity().getQualifiedUser())); + } + trow.addToColumnValue(new TCell().setStringVal("")); + trow.addToColumnValue(new TCell().setStringVal(runningOffset == null + ? FeConstants.null_string : runningOffset.showRange())); + return trow; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 3a8ee70630adac..28d8fa5ec965ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -35,6 +35,7 @@ import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.base.TimerDefinition; +import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.common.FailureReason; import org.apache.doris.job.common.IntervalUnit; import org.apache.doris.job.common.JobStatus; @@ -47,6 +48,8 @@ import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; import org.apache.doris.job.offset.SourceOffsetProviderFactory; +import org.apache.doris.job.offset.jdbc.JdbcSourceOffsetProvider; +import org.apache.doris.job.util.StreamingJobUtils; import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.loadv2.LoadStatistic; import org.apache.doris.mysql.privilege.PrivPredicate; @@ -55,25 +58,17 @@ import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.AlterJobCommand; import org.apache.doris.nereids.trees.plans.commands.info.BaseViewInfo; +import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertUtils; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; -import org.apache.doris.proto.InternalService; -import org.apache.doris.proto.InternalService.PRequestCdcClientResult; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; -import org.apache.doris.resource.Tag; -import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.rpc.RpcException; -import org.apache.doris.system.Backend; -import org.apache.doris.system.BeSelectionPolicy; -import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TCell; -import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TRow; -import org.apache.doris.thrift.TStatusCode; import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnStateChangeCallback; @@ -85,21 +80,24 @@ import lombok.extern.log4j.Log4j2; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; - +import static org.apache.doris.job.common.LoadConstants.DATABASE; +import static org.apache.doris.job.common.LoadConstants.DRIVER_CLASS; +import static org.apache.doris.job.common.LoadConstants.DRIVER_URL; +import static org.apache.doris.job.common.LoadConstants.EXCLUDE_TABLES; +import static org.apache.doris.job.common.LoadConstants.INCLUDE_TABLES; +import static org.apache.doris.job.common.LoadConstants.JDBC_URL; +import static org.apache.doris.job.common.LoadConstants.PASSWORD; +import static org.apache.doris.job.common.LoadConstants.USER; import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -127,18 +125,31 @@ public class StreamingInsertJob extends AbstractJob originTvfProps; @Getter - StreamingInsertTask runningStreamTask; + AbstractStreamingTask runningStreamTask; SourceOffsetProvider offsetProvider; @Setter @Getter private long lastScheduleTaskTimestamp = -1L; private InsertIntoTableCommand baseCommand; private ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); - private ConcurrentLinkedQueue streamInsertTaskQueue = new ConcurrentLinkedQueue<>(); + private ConcurrentLinkedQueue streamInsertTaskQueue = new ConcurrentLinkedQueue<>(); @Setter @Getter private String jobRuntimeMsg = ""; + @Getter + @SerializedName("tdb") + private String targetDb; + @Getter + @SerializedName("ds") + private DataSourceType dataSourceType; + @Getter + @SerializedName("sprops") + private Map sourceProperties; + @Getter + @SerializedName("tprops") + private Map targetProperties; + public StreamingInsertJob(String jobName, JobStatus jobStatus, String dbName, @@ -151,74 +162,80 @@ public StreamingInsertJob(String jobName, super(Env.getCurrentEnv().getNextId(), jobName, jobStatus, dbName, comment, createUser, jobConfig, createTimeMs, executeSql); this.properties = properties; + initInsertJob(); + } + + // for streaming job from...to database + public StreamingInsertJob(String jobName, + JobStatus jobStatus, + String dbName, + String comment, + UserIdentity createUser, + JobExecutionConfiguration jobConfig, + Long createTimeMs, + String executeSql, + Map properties, + String targetDb, + DataSourceType dataSourceType, + Map sourceProperties, + Map targetProperties) { + super(Env.getCurrentEnv().getNextId(), jobName, jobStatus, dbName, comment, createUser, + jobConfig, createTimeMs, executeSql); + this.properties = properties; + this.targetDb = targetDb; + this.dataSourceType = dataSourceType; + this.sourceProperties = sourceProperties; + this.targetProperties = targetProperties; + initSourceJob(); + } + + /** + * Initialize job from source to database, like multi table mysql to doris. + * 1. get mysql connection info from sourceProperties + * 2. fetch table list from mysql + * 3. create doris table if not exists + * 4. check whether need full data sync + * 5. need => fetch split and write to system table + */ + private void initSourceJob() { try { - log.info("======fetch cdc split"); - fetchCdcSplit(); - } catch (JobException e) { - throw new RuntimeException(e); - } - init(); - } - - // todo: down to offset provider - private void fetchCdcSplit() throws JobException{ - Backend backend = selectBackend(1L); - Map params = new HashMap<>(); - params.put("jobId", "1"); - params.put("dataSource", "MYSQL"); - params.put("snapshotTable", "user_info"); - Map config = new HashMap<>(); - config.put("host", "127.0.0.1"); - config.put("port", "3308"); - config.put("username", "root"); - config.put("password", "123456"); - config.put("database", "test"); - config.put("include_tables", "user_info"); - params.put("config", config); - - InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() - .setApi("/api/fetchSplits") - .setParams(GsonUtils.GSON.toJson(params)).build(); - TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); - InternalService.PRequestCdcClientResult result = null; - try { - Future future = - BackendServiceProxy.getInstance().requestCdcClient(address, request); - result = future.get(); - TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); - if (code != TStatusCode.OK) { - log.error("Failed to get split from backend, {}", result.getStatus().getErrorMsgs(0)); - throw new JobException("Failed to get split from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + result.getResponse()); - } - } catch (ExecutionException | InterruptedException ex) { - log.error("Get splits error: ", ex); - throw new JobException(ex); + init(); + checkRequiredSourceProperties(); + List createTbls = createTableIfNotExists(); + this.offsetProvider = new JdbcSourceOffsetProvider(); + JdbcSourceOffsetProvider rdsOffsetProvider = (JdbcSourceOffsetProvider)this.offsetProvider; + rdsOffsetProvider.setJobId(getJobId()); + rdsOffsetProvider.setSourceType(dataSourceType); + rdsOffsetProvider.setSourceProperties(sourceProperties); + rdsOffsetProvider.splitChunks(createTbls); + }catch (Exception ex){ + log.warn("init streaming job for {} failed", dataSourceType, ex); + throw new RuntimeException(ex.getMessage()); } - log.info("========fetch cdc split {}", result.getResponse()); } - public static Backend selectBackend(Long jobId) throws JobException { - Backend backend = null; - BeSelectionPolicy policy = null; + private void checkRequiredSourceProperties() { + Preconditions.checkArgument(sourceProperties.get(JDBC_URL) != null, "jdbc_url is required property"); + Preconditions.checkArgument(sourceProperties.get(DRIVER_URL) != null, "driver_url is required property"); + Preconditions.checkArgument(sourceProperties.get(DRIVER_CLASS) != null, "driver_class is required property"); + Preconditions.checkArgument(sourceProperties.get(USER) != null, "user is required property"); + Preconditions.checkArgument(sourceProperties.get(PASSWORD) != null, "password is required property"); + Preconditions.checkArgument(sourceProperties.get(DATABASE) != null, "database is required property"); + Preconditions.checkArgument(sourceProperties.get(INCLUDE_TABLES) != null || sourceProperties.get(EXCLUDE_TABLES) != null, + "Either include_tables or exclude_tables must be specified"); + } - policy = new BeSelectionPolicy.Builder() - .setEnableRoundRobin(true) - .needLoadAvailable().build(); - List backendIds; - backendIds = Env.getCurrentSystemInfo().selectBackendIdsByPolicy(policy, 1); - if (backendIds.isEmpty()) { - throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); - } - //jobid % backendSize - long index = backendIds.get(jobId.intValue() % backendIds.size()); - backend = Env.getCurrentSystemInfo().getBackend(index); - if (backend == null) { - throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); + private List createTableIfNotExists() throws UserException, JobException { + List syncTbls = new ArrayList<>(); + List createTableInfos = StreamingJobUtils.generateCreateTableInfos(targetDb, + dataSourceType, sourceProperties, targetProperties); + for (CreateTableInfo createTableInfo : createTableInfos) { + Env.getCurrentEnv().createTable(createTableInfo); + syncTbls.add(createTableInfo.getTableName()); } - return backend; + return syncTbls; } - private void init() { try { this.jobProperties = new StreamingJobProperties(properties); @@ -230,7 +247,15 @@ private void init() { timerDefinition.setIntervalUnit(IntervalUnit.SECOND); timerDefinition.setStartTimeMs(execConfig.getTimerDefinition().getStartTimeMs()); execConfig.setTimerDefinition(timerDefinition); + } catch (AnalysisException ae) { + log.warn("parse streaming insert job failed, props: {}", properties, ae); + throw new RuntimeException(ae.getMessage()); + } + } + private void initInsertJob() { + try { + init(); UnboundTVFRelation currentTvf = getCurrentTvf(); this.tvfType = currentTvf.getFunctionName(); this.originTvfProps = currentTvf.getProperties().getMap(); @@ -240,9 +265,6 @@ private void init() { Offset offset = validateOffset(jobProperties.getOffsetProperty()); this.offsetProvider.updateOffset(offset); } - } catch (AnalysisException ae) { - log.warn("parse streaming insert job failed, props: {}", properties, ae); - throw new RuntimeException(ae.getMessage()); } catch (Exception ex) { log.warn("init streaming insert job failed, sql: {}", getExecuteSql(), ex); throw new RuntimeException(ex.getMessage()); @@ -394,12 +416,13 @@ public List createTasks(TaskType taskType, Map queryAllStreamTasks() { + public List queryAllStreamTasks() { if (CollectionUtils.isEmpty(streamInsertTaskQueue)) { return new ArrayList<>(); } - List tasks = new ArrayList<>(streamInsertTaskQueue); - Comparator taskComparator = - Comparator.comparingLong(StreamingInsertTask::getCreateTimeMs).reversed(); + List tasks = new ArrayList<>(streamInsertTaskQueue); + Comparator taskComparator = + Comparator.comparingLong(AbstractStreamingTask::getCreateTimeMs).reversed(); tasks.sort(taskComparator); return tasks; } protected void fetchMeta() { try { - if (originTvfProps == null) { + if (originTvfProps == null && StringUtils.isNotEmpty(getExecuteSql())) { this.originTvfProps = getCurrentTvf().getProperties().getMap(); + offsetProvider.fetchRemoteMeta(originTvfProps); + } else { + offsetProvider.fetchRemoteMeta(new HashMap<>()); } - offsetProvider.fetchRemoteMeta(originTvfProps); } catch (Exception ex) { log.warn("fetch remote meta failed, job id: {}", getJobId(), ex); failureReason = new FailureReason(InternalErrorCode.GET_REMOTE_DATA_ERROR, @@ -499,7 +542,7 @@ public void onStreamTaskSuccess(StreamingInsertTask task) { try { succeedTaskCount.incrementAndGet(); Env.getCurrentEnv().getJobManager().getStreamingTaskManager().removeRunningTask(task); - StreamingInsertTask nextTask = createStreamingInsertTask(); + AbstractStreamingTask nextTask = createStreamingInsertTask(); this.runningStreamTask = nextTask; log.info("Streaming insert job {} create next streaming insert task {} after task {} success", getJobId(), nextTask.getTaskId(), task.getTaskId()); @@ -624,9 +667,7 @@ public TRow getTvfInfo() { trow.addToColumnValue(new TCell().setStringVal(getJobConfig().getExecuteType().name())); trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); trow.addToColumnValue(new TCell().setStringVal(getJobStatus().name())); - - trow.addToColumnValue(new TCell().setStringVal(StringUtils.isNotEmpty(getEncryptedSql()) - ? getEncryptedSql() : generateEncryptedSql())); + trow.addToColumnValue(new TCell().setStringVal(getShowSQL())); trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getCreateTimeMs()))); trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getSucceedTaskCount().get()))); trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getFailedTaskCount().get()))); @@ -656,7 +697,35 @@ public TRow getTvfInfo() { return trow; } - private static boolean checkPrivilege(ConnectContext ctx, LogicalPlan logicalPlan) throws AnalysisException { + private String getShowSQL() { + if (StringUtils.isNotEmpty(getExecuteSql())) { + return StringUtils.isNotEmpty(getEncryptedSql()) + ? getEncryptedSql() : generateEncryptedSql(); + } else { + StringBuilder sb = new StringBuilder(); + sb.append("FROM ").append(dataSourceType.name()); + sb.append("("); + for (Map.Entry entry : sourceProperties.entrySet()) { + sb.append("'").append(entry.getKey()) + .append("'='").append(entry.getValue()).append("',"); + } + sb.deleteCharAt(sb.length() - 1); + sb.append(" ) TO DATABSE ").append(targetDb); + if (!targetProperties.isEmpty()) { + sb.append(" ("); + for (Map.Entry entry : targetProperties.entrySet()) { + sb.append("'").append(entry.getKey()) + .append("'='").append(entry.getValue()).append("',"); + } + sb.deleteCharAt(sb.length() - 1); + sb.append(")"); + } + return sb.toString(); + } + } + + private static boolean checkPrivilege(ConnectContext ctx, String sql) throws AnalysisException { + LogicalPlan logicalPlan = new NereidsParser().parseSingle(sql); if (!(logicalPlan instanceof InsertIntoTableCommand)) { throw new AnalysisException("Only support insert command"); } @@ -677,33 +746,59 @@ private static boolean checkPrivilege(ConnectContext ctx, LogicalPlan logicalPla return true; } - public boolean checkPrivilege(ConnectContext ctx) throws AnalysisException { - LogicalPlan logicalPlan = new NereidsParser().parseSingle(getExecuteSql()); - return checkPrivilege(ctx, logicalPlan); + public static boolean checkPrivilege(ConnectContext ctx, String sql, String targetDb) throws AnalysisException { + if (StringUtils.isNotEmpty(sql)) { + return checkPrivilege(ctx, sql); + }else if (StringUtils.isNotEmpty(targetDb)) { + return checkHasSourceJobPriv(ctx, targetDb); + } else { + log.info("insert sql and target db are both empty"); + return false; + } } - public static boolean checkPrivilege(ConnectContext ctx, String sql) throws AnalysisException { - LogicalPlan logicalPlan = new NereidsParser().parseSingle(sql); - return checkPrivilege(ctx, logicalPlan); + public boolean checkPrivilege(ConnectContext ctx) throws AnalysisException { + if (StringUtils.isNotEmpty(getExecuteSql())) { + return checkPrivilege(ctx, getExecuteSql()); + }else if (StringUtils.isNotEmpty(getTargetDb())){ + return checkHasSourceJobPriv(ctx, targetDb); + }else { + log.info("insert sql and target db are both empty"); + return false; + } } public boolean hasPrivilege(UserIdentity userIdentity) { ConnectContext ctx = InsertTask.makeConnectContext(userIdentity, getCurrentDbName()); try { - LogicalPlan logicalPlan = new NereidsParser().parseSingle(getExecuteSql()); - LogicalPlan logicalQuery = ((InsertIntoTableCommand) logicalPlan).getLogicalQuery(); - List targetTable = InsertUtils.getTargetTableQualified(logicalQuery, ctx); - Preconditions.checkArgument(targetTable.size() == 3, "target table name is invalid"); - return Env.getCurrentEnv().getAccessManager().checkTblPriv(userIdentity, - InternalCatalog.INTERNAL_CATALOG_NAME, - targetTable.get(1), - targetTable.get(2), - PrivPredicate.LOAD); + if (StringUtils.isNotEmpty(getExecuteSql())) { + LogicalPlan logicalPlan = new NereidsParser().parseSingle(getExecuteSql()); + LogicalPlan logicalQuery = ((InsertIntoTableCommand) logicalPlan).getLogicalQuery(); + List targetTable = InsertUtils.getTargetTableQualified(logicalQuery, ctx); + Preconditions.checkArgument(targetTable.size() == 3, "target table name is invalid"); + return Env.getCurrentEnv().getAccessManager().checkTblPriv(userIdentity, + InternalCatalog.INTERNAL_CATALOG_NAME, + targetTable.get(1), + targetTable.get(2), + PrivPredicate.LOAD); + } else if (StringUtils.isNotEmpty(getTargetDb())){ + return checkHasSourceJobPriv(ctx, targetDb); + } else { + log.info("insert sql and target db are both empty"); + return false; + } } finally { ctx.cleanup(); } } + private static boolean checkHasSourceJobPriv(ConnectContext ctx, String targetDb){ + return Env.getCurrentEnv().getAccessManager().checkDbPriv(ctx, + InternalCatalog.INTERNAL_CATALOG_NAME, + targetDb, + PrivPredicate.LOAD); + } + private String generateEncryptedSql() { makeConnectContext(); TreeMap, String> indexInSqlToString = new TreeMap<>(new Pair.PairComparator<>()); @@ -859,8 +954,12 @@ public void replayOnVisible(TransactionState txnState) { @Override public void gsonPostProcess() throws IOException { - if (offsetProvider == null && tvfType != null) { - offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(tvfType); + if (offsetProvider == null) { + if (tvfType != null) { + offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(tvfType); + }else { + offsetProvider = new JdbcSourceOffsetProvider(); + } } if (jobProperties == null && properties != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java index f25ebe794a0788..bdbe188e786230 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java @@ -54,31 +54,16 @@ @Log4j2 @Getter -public class StreamingInsertTask { +public class StreamingInsertTask extends AbstractStreamingTask { private static final String LABEL_SPLITTER = "_"; - private static final int MAX_RETRY = 3; - private long jobId; - private long taskId; - private String labelName; - @Setter - private volatile TaskStatus status; - private String errMsg; - private Long createTimeMs; - private Long startTimeMs; - private Long finishTimeMs; private String sql; private StmtExecutor stmtExecutor; private InsertIntoTableCommand taskCommand; private String currentDb; - private UserIdentity userIdentity; private ConnectContext ctx; - private Offset runningOffset; - @Getter - private AtomicBoolean isCanceled = new AtomicBoolean(false); private StreamingJobProperties jobProperties; private Map originTvfProps; SourceOffsetProvider offsetProvider; - private int retryCount = 0; public StreamingInsertTask(long jobId, long taskId, @@ -100,39 +85,9 @@ public StreamingInsertTask(long jobId, this.createTimeMs = System.currentTimeMillis(); } - public void execute() throws JobException { - while (retryCount <= MAX_RETRY) { - try { - before(); - run(); - onSuccess(); - return; - } catch (Exception e) { - if (TaskStatus.CANCELED.equals(status)) { - return; - } - this.errMsg = e.getMessage(); - retryCount++; - if (retryCount > MAX_RETRY) { - log.error("Task execution failed after {} retries.", MAX_RETRY, e); - onFail(e.getMessage()); - return; - } - log.warn("execute streaming task error, job id is {}, task id is {}, retrying {}/{}: {}", - jobId, taskId, retryCount, MAX_RETRY, e.getMessage()); - } finally { - // The cancel logic will call the closeOrReleased Resources method by itself. - // If it is also called here, - // it may result in the inability to obtain relevant information when canceling the task - if (!TaskStatus.CANCELED.equals(status)) { - closeOrReleaseResources(); - } - } - } - } - - private void before() throws Exception { - if (isCanceled.get()) { + @Override + public void before() throws Exception { + if (getIsCanceled().get()) { log.info("streaming insert task has been canceled, task id is {}", getTaskId()); return; } @@ -158,8 +113,9 @@ private void before() throws Exception { this.stmtExecutor = new StmtExecutor(ctx, new LogicalPlanAdapter(taskCommand, ctx.getStatementContext())); } - private void run() throws JobException { - if (isCanceled.get()) { + @Override + public void run() throws JobException { + if (getIsCanceled().get()) { log.info("task has been canceled, task id is {}", getTaskId()); return; } @@ -180,8 +136,9 @@ private void run() throws JobException { } } + @Override public boolean onSuccess() throws JobException { - if (isCanceled.get()) { + if (getIsCanceled().get()) { return false; } this.status = TaskStatus.SUCCESS; @@ -200,8 +157,9 @@ public boolean onSuccess() throws JobException { return true; } + @Override public void onFail(String errMsg) throws JobException { - if (isCanceled.get()) { + if (getIsCanceled().get()) { return; } this.errMsg = errMsg; @@ -215,16 +173,17 @@ public void onFail(String errMsg) throws JobException { streamingInsertJob.onStreamTaskFail(this); } + @Override public void cancel(boolean needWaitCancelComplete) { if (TaskStatus.SUCCESS.equals(status) || TaskStatus.FAILED.equals(status) || TaskStatus.CANCELED.equals(status)) { return; } status = TaskStatus.CANCELED; - if (isCanceled.get()) { + if (getIsCanceled().get()) { return; } - isCanceled.getAndSet(true); + getIsCanceled().getAndSet(true); this.errMsg = "task cancelled"; if (null != stmtExecutor) { log.info("cancelling streaming insert task, job id is {}, task id is {}", @@ -234,6 +193,7 @@ public void cancel(boolean needWaitCancelComplete) { } } + @Override public void closeOrReleaseResources() { if (null != stmtExecutor) { stmtExecutor = null; @@ -255,56 +215,4 @@ private boolean isCallable() { } return false; } - - /** - * show streaming insert task info detail - */ - public TRow getTvfInfo(String jobName) { - TRow trow = new TRow(); - trow.addToColumnValue(new TCell().setStringVal(String.valueOf(this.getTaskId()))); - trow.addToColumnValue(new TCell().setStringVal(String.valueOf(this.getJobId()))); - trow.addToColumnValue(new TCell().setStringVal(jobName)); - trow.addToColumnValue(new TCell().setStringVal(this.getLabelName())); - trow.addToColumnValue(new TCell().setStringVal(this.getStatus().name())); - // err msg - trow.addToColumnValue(new TCell().setStringVal(StringUtils.isNotBlank(errMsg) - ? errMsg : FeConstants.null_string)); - - // create time - trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(this.getCreateTimeMs()))); - trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? FeConstants.null_string - : TimeUtils.longToTimeString(this.getStartTimeMs()))); - // load end time - trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(this.getFinishTimeMs()))); - - List loadJobs = Env.getCurrentEnv().getLoadManager() - .queryLoadJobsByJobIds(Arrays.asList(this.getTaskId())); - if (!loadJobs.isEmpty()) { - LoadJob loadJob = loadJobs.get(0); - if (loadJob.getLoadingStatus() != null && loadJob.getLoadingStatus().getTrackingUrl() != null) { - trow.addToColumnValue(new TCell().setStringVal(loadJob.getLoadingStatus().getTrackingUrl())); - } else { - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - } - - if (loadJob.getLoadStatistic() != null) { - trow.addToColumnValue(new TCell().setStringVal(loadJob.getLoadStatistic().toJson())); - } else { - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - } - } else { - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - } - - if (this.getUserIdentity() == null) { - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - } else { - trow.addToColumnValue(new TCell().setStringVal(this.getUserIdentity().getQualifiedUser())); - } - trow.addToColumnValue(new TCell().setStringVal("")); - trow.addToColumnValue(new TCell().setStringVal(runningOffset == null - ? FeConstants.null_string : runningOffset.showRange())); - return trow; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java index c3655e6697eb9f..20d030534d586a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -65,7 +65,8 @@ private void handlePendingState() throws JobException { return; } } - streamingInsertJob.createStreamingInsertTask(); + + streamingInsertJob.createStreamingTask();; streamingInsertJob.updateJobStatus(JobStatus.RUNNING); streamingInsertJob.setAutoResumeCount(0); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java new file mode 100644 index 00000000000000..089c3553b4f3b5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java @@ -0,0 +1,61 @@ +package org.apache.doris.job.extensions.insert.streaming; + +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.offset.SourceOffsetProvider; + +import lombok.Getter; +import lombok.extern.log4j.Log4j2; +import java.util.Map; + +@Log4j2 +@Getter +public class StreamingMultiTblTask extends AbstractStreamingTask { + + SourceOffsetProvider offsetProvider; + Map sourceProperties; + Map targetProperties; + + public StreamingMultiTblTask(Long jobId, + long taskId, + SourceOffsetProvider offsetProvider, + Map sourceProperties, + Map targetProperties) { + this.jobId = jobId; + this.taskId = taskId; + this.offsetProvider = offsetProvider; + this.sourceProperties = sourceProperties; + this.targetProperties = targetProperties; + } + + @Override + public void before() throws Exception { + log.info("StreamingMultiTblTask before execution."); + this.runningOffset = offsetProvider.getNextOffset(null, sourceProperties); + } + + @Override + public void run() throws JobException { + log.info("StreamingMultiTblTask run execution."); + } + + @Override + public boolean onSuccess() throws JobException { + log.info("StreamingMultiTblTask onSuccess execution."); + return false; + } + + @Override + public void onFail(String errMsg) throws JobException { + + } + + @Override + public void cancel(boolean needWaitCancelComplete) { + + } + + @Override + public void closeOrReleaseResources() { + + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java index 7c6fecee14a179..afe4e62baaadc8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java @@ -17,6 +17,7 @@ package org.apache.doris.job.manager; +import org.apache.doris.job.extensions.insert.streaming.AbstractStreamingTask; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; import lombok.Getter; @@ -28,28 +29,29 @@ public class StreamingTaskManager { @Getter - private final LinkedBlockingDeque needScheduleTasksQueue = new LinkedBlockingDeque<>(); + private final LinkedBlockingDeque needScheduleTasksQueue = new LinkedBlockingDeque<>(); @Getter - private List runningTasks = Collections.synchronizedList(new ArrayList<>()); + private List runningTasks = Collections.synchronizedList(new ArrayList<>()); - public void registerTask(StreamingInsertTask task) { + public void registerTask(AbstractStreamingTask task) { needScheduleTasksQueue.add(task); } public StreamingInsertTask getStreamingInsertTaskById(long taskId) { synchronized (runningTasks) { - return runningTasks.stream() + return (StreamingInsertTask) runningTasks.stream() .filter(task -> task.getTaskId() == taskId) + .filter(task -> task instanceof StreamingInsertTask) .findFirst() .orElse(null); } } - public void addRunningTask(StreamingInsertTask task) { + public void addRunningTask(AbstractStreamingTask task) { runningTasks.add(task); } - public void removeRunningTask(StreamingInsertTask task) { + public void removeRunningTask(AbstractStreamingTask task) { runningTasks.remove(task); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java new file mode 100644 index 00000000000000..6545a1969bdd66 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -0,0 +1,34 @@ +package org.apache.doris.job.offset.jdbc; + +import org.apache.doris.job.offset.Offset; +import org.apache.doris.job.offset.jdbc.split.AbstractSourceSplit; +import lombok.Getter; +import lombok.Setter; +import java.util.Map; + +@Getter +@Setter +public class JdbcOffset implements Offset { + + private Map meta; + + @Override + public String toSerializedJson() { + return null; + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public boolean isValidOffset() { + return false; + } + + @Override + public String showRange() { + return null; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java new file mode 100644 index 00000000000000..6a6e87b1626d3a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -0,0 +1,231 @@ +package org.apache.doris.job.offset.jdbc; + +import org.apache.doris.catalog.Env; +import org.apache.doris.httpv2.entity.ResponseBody; +import org.apache.doris.job.common.DataSourceType; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; +import org.apache.doris.job.offset.Offset; +import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.job.offset.jdbc.split.SnapshotSplit; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.system.Backend; +import org.apache.doris.system.BeSelectionPolicy; +import org.apache.doris.system.SystemInfoService; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.Getter; +import lombok.Setter; +import lombok.extern.log4j.Log4j2; +import static org.apache.doris.job.common.LoadConstants.STARTUP_MODE; +import static org.apache.doris.job.util.StreamingJobUtils.createMetaTableIfNotExist; +import static org.apache.doris.job.util.StreamingJobUtils.insertSplitsToMeta; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.util.EntityUtils; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +@Getter +@Setter +@Log4j2 +public class JdbcSourceOffsetProvider implements SourceOffsetProvider { + private static final ObjectMapper objectMapper = new ObjectMapper(); + private Long jobId; + private DataSourceType sourceType; + private Map sourceProperties = new HashMap<>(); + + List remainingSplits; + List assignedSplits; + + JdbcOffset currentOffset; + Map endBinlogOffset; + + @Override + public String getSourceType() { + return "jdbc"; + } + + @Override + public Offset getNextOffset(StreamingJobProperties jobProps, Map properties) { + // 全量从系统表中获取 + + // 增量直接就是上一次的offset + return null; + } + + @Override + public String getShowCurrentOffset() { + return null; + } + + @Override + public String getShowMaxOffset() { + return null; + } + + @Override + public InsertIntoTableCommand rewriteTvfParams(InsertIntoTableCommand originCommand, Offset nextOffset) { + // not need + return null; + } + + @Override + public void updateOffset(Offset offset) { + + } + + @Override + public void fetchRemoteMeta(Map properties) throws Exception { + // todo: request cdc client api + } + + @Override + public boolean hasMoreDataToConsume() { + return false; + } + + @Override + public Offset deserializeOffset(String offset) { + return null; + } + + @Override + public Offset deserializeOffsetProperty(String offset) { + return null; + } + + public void splitChunks(List createTbls) throws JobException { + // todo: When splitting takes a long time, it needs to be changed to asynchronous. + if(checkNeedSplitChunks(sourceProperties)) { + Map> tableSplits = new HashMap<>(); + for (String tbl : createTbls) { + List snapshotSplits = requestTableSplits(tbl); + tableSplits.put(tbl, snapshotSplits); + } + // save chunk list to system table + saveChunkMeta(tableSplits); + this.remainingSplits = tableSplits.values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); + } + } + + private void saveChunkMeta(Map> tableSplits) throws JobException { + try { + createMetaTableIfNotExist(); + insertSplitsToMeta(getJobId(), tableSplits); + } catch (Exception e) { + log.warn("save chunk meta error: ", e); + throw new JobException(e.getMessage()); + } + } + + private List requestTableSplits(String table) throws JobException { + Map params = buildSplitParams(table); + String url = "http://127.0.0.1:9096/api/fetchSplits"; + // Prepare request body + String requestBody = GsonUtils.GSON.toJson(params); + + // Create HTTP POST request + HttpPost httpPost = new HttpPost(url); + StringEntity stringEntity = new StringEntity(requestBody, "UTF-8"); + stringEntity.setContentType("application/json"); + httpPost.setEntity(stringEntity); + // Set request headers + httpPost.setHeader("Content-Type", "application/json"); + + // Execute request + try (CloseableHttpClient client = HttpClientBuilder.create().build()) { + String response = client.execute(httpPost, httpResponse -> { + int statusCode = httpResponse.getStatusLine().getStatusCode(); + String responseBody = EntityUtils.toString(httpResponse.getEntity(), "UTF-8"); + if (statusCode != 200) { + throw new RuntimeException("Failed to get split from CDC client, HTTP status code: " + statusCode); + } + return responseBody; + }); + + ResponseBody> responseObj = objectMapper.readValue( + response, + new TypeReference>>() {} + ); + List splits = responseObj.getData(); + return splits; + + } catch (Exception ex) { + log.error("Get splits error: ", ex); + throw new JobException("Failed to request CDC client: " + ex.getMessage(), ex); + } + + /** + Backend backend = selectBackend(jobId); + Map params = buildSplitParams(table); + InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() + .setApi("/api/fetchSplits") + .setParams(GsonUtils.GSON.toJson(params)).build(); + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); + InternalService.PRequestCdcClientResult result = null; + try { + Future future = + BackendServiceProxy.getInstance().requestCdcClient(address, request); + result = future.get(); + TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + log.error("Failed to get split from backend, {}", result.getStatus().getErrorMsgs(0)); + throw new JobException("Failed to get split from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + result.getResponse()); + } + } catch (ExecutionException | InterruptedException ex) { + log.error("Get splits error: ", ex); + throw new JobException(ex); + } + log.info("========fetch cdc split {}", result.getResponse()); + return ""; + **/ + } + + private Map buildSplitParams(String table) { + Map params = new HashMap<>(); + params.put("jobId", getJobId()); + params.put("dataSource", sourceType); + params.put("snapshotTable", table); + params.put("config", sourceProperties); + return params; + } + + public static Backend selectBackend(Long jobId) throws JobException { + Backend backend = null; + BeSelectionPolicy policy = null; + + policy = new BeSelectionPolicy.Builder() + .setEnableRoundRobin(true) + .needLoadAvailable().build(); + List backendIds; + backendIds = Env.getCurrentSystemInfo().selectBackendIdsByPolicy(policy, 1); + if (backendIds.isEmpty()) { + throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); + } + // jobid % backendSize + long index = backendIds.get(jobId.intValue() % backendIds.size()); + backend = Env.getCurrentSystemInfo().getBackend(index); + if (backend == null) { + throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); + } + return backend; + } + + private boolean checkNeedSplitChunks(Map sourceProperties) { + String startMode = sourceProperties.get(STARTUP_MODE); + if (startMode == null) { + return false; + } + return "snapshot".equalsIgnoreCase(startMode) || "initial".equalsIgnoreCase(startMode); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java new file mode 100644 index 00000000000000..16e653fce9beef --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java @@ -0,0 +1,32 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.offset.jdbc.split; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; + +@Getter +@Setter +@AllArgsConstructor +@NoArgsConstructor +public abstract class AbstractSourceSplit implements SourceSplit { + private static final long serialVersionUID = 1L; + protected String splitId; +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java new file mode 100644 index 00000000000000..fff821805ede53 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java @@ -0,0 +1,36 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.offset.jdbc.split; + +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.Setter; +import java.util.Map; + +@Getter +@Setter +@EqualsAndHashCode(callSuper = true) +public class BinlogSplit extends AbstractSourceSplit { + private static final long serialVersionUID = 1L; + private Map offset; + + public BinlogSplit(String splitId, Map offset) { + super(splitId); + this.offset = offset; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java new file mode 100644 index 00000000000000..b037c47f4c5b65 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java @@ -0,0 +1,103 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.offset.jdbc.split; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +@Getter +@Setter +@EqualsAndHashCode(callSuper = true) +@NoArgsConstructor +public class SnapshotSplit extends AbstractSourceSplit { + private static final long serialVersionUID = 1L; + private static ObjectMapper objectMapper = new ObjectMapper(); + private String tableId; + private List splitKey; + private Object[] splitStart; + private Object[] splitEnd; + private Map highWatermark; + + public SnapshotSplit( + String splitId, + String tableId, + List splitKey, + Object[] splitStart, + Object[] splitEnd, + Map highWatermark) { + super(splitId); + this.tableId = tableId; + this.splitKey = splitKey; + this.splitStart = splitStart; + this.splitEnd = splitEnd; + this.highWatermark = highWatermark; + } + + public static SnapshotSplit fromMap(Map map) throws JsonProcessingException { + if (map == null || map.isEmpty()) { + return null; + } + + SnapshotSplit split = new SnapshotSplit(); + String splitId = map.get("splitId"); + String tableId = map.get("tableId"); + String splitKeyStr = map.get("splitKey"); + Preconditions.checkNotNull(splitKeyStr, "splitKey must not be null"); + List splitKey = + objectMapper.readValue(splitKeyStr, new TypeReference>() {}); + + split.setSplitId(splitId); + split.setTableId(tableId); + split.setSplitKey(splitKey); + + String splitStartStr = map.get("splitStart"); + if (splitStartStr != null) { + Object[] splitStart = objectMapper.readValue(splitStartStr, Object[].class); + split.setSplitStart(splitStart); + } + + String splitEndStr = map.get("splitEnd"); + if (splitEndStr != null) { + Object[] splitEnd = objectMapper.readValue(splitEndStr, Object[].class); + split.setSplitEnd(splitEnd); + } + + String highWatermarkStr = map.get("highWatermark"); + if (highWatermarkStr != null) { + Map highWatermark = + objectMapper.readValue( + highWatermarkStr, new TypeReference>() {}); + split.setHighWatermark(highWatermark); + } + + return split; + } + + public static String getOrEmptyArray(Map map, String key) { + return Optional.ofNullable(map.get(key)).orElse("[]"); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SourceSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SourceSplit.java new file mode 100644 index 00000000000000..5bf6f2c1b2bf70 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SourceSplit.java @@ -0,0 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.offset.jdbc.split; + +public interface SourceSplit {} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java index ea1cf7825ec56b..b19839ef646ad4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java @@ -25,6 +25,7 @@ import org.apache.doris.job.common.FailureReason; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.AbstractStreamingTask; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; @@ -74,16 +75,16 @@ protected void runAfterCatalogReady() { } private void process() throws InterruptedException { - List tasks = new ArrayList<>(); - LinkedBlockingDeque needScheduleTasksQueue = + List tasks = new ArrayList<>(); + LinkedBlockingDeque needScheduleTasksQueue = Env.getCurrentEnv().getJobManager().getStreamingTaskManager().getNeedScheduleTasksQueue(); tasks.add(needScheduleTasksQueue.take()); needScheduleTasksQueue.drainTo(tasks); scheduleTasks(tasks); } - private void scheduleTasks(List tasks) { - for (StreamingInsertTask task : tasks) { + private void scheduleTasks(List tasks) { + for (AbstractStreamingTask task : tasks) { threadPool.execute(() -> { try { scheduleOneTask(task); @@ -104,7 +105,7 @@ private void scheduleTasks(List tasks) { } } - private void scheduleOneTask(StreamingInsertTask task) { + private void scheduleOneTask(AbstractStreamingTask task) { if (DebugPointUtil.isEnable("StreamingJob.scheduleTask.exception")) { throw new RuntimeException("debug point StreamingJob.scheduleTask.exception"); } @@ -143,7 +144,7 @@ private void scheduleOneTask(StreamingInsertTask task) { } } - private void scheduleTaskWithDelay(StreamingInsertTask task, long delayMs) { + private void scheduleTaskWithDelay(AbstractStreamingTask task, long delayMs) { delayScheduler.schedule(() -> { Env.getCurrentEnv().getJobManager().getStreamingTaskManager().registerTask(task); }, delayMs, TimeUnit.MILLISECONDS); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java new file mode 100644 index 00000000000000..33589827361a06 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java @@ -0,0 +1,256 @@ +package org.apache.doris.job.util; + +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.AggregateType; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.FeConstants; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.datasource.jdbc.client.JdbcClient; +import org.apache.doris.datasource.jdbc.client.JdbcClientConfig; +import org.apache.doris.datasource.jdbc.client.JdbcMySQLClient; +import org.apache.doris.job.common.DataSourceType; +import org.apache.doris.job.common.LoadConstants; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.offset.jdbc.split.SnapshotSplit; +import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; +import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; +import org.apache.doris.nereids.trees.plans.commands.info.DistributionDescriptor; +import org.apache.doris.nereids.trees.plans.commands.info.PartitionTableInfo; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.qe.AutoCloseConnectContext; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.SessionVariable; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.thrift.TUniqueId; + +import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.extern.log4j.Log4j2; +import org.apache.commons.text.StringSubstitutor; +import static org.apache.doris.job.common.LoadConstants.DRIVER_URL; +import static org.apache.doris.job.common.LoadConstants.*; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.stream.Collectors; + +@Log4j2 +public class StreamingJobUtils { + public static final String INTERNAL_STREAMING_JOB_META_TABLE_NAME = "streaming_job_meta"; + public static final String FULL_QUALIFIED_META_TBL_NAME = InternalCatalog.INTERNAL_CATALOG_NAME + + "." + FeConstants.INTERNAL_DB_NAME + "." + INTERNAL_STREAMING_JOB_META_TABLE_NAME; + private static final String CREATE_META_TABLE = "CREATE TABLE %s(\n" + + "id varchar(32),\n" + + "job_id varchar(256),\n" + + "table_name string,\n" + + "chunk_list json\n" + + ")\n" + + "UNIQUE KEY(id)\n" + + "DISTRIBUTED BY HASH(id)\n" + + "BUCKETS 1\n" + + "PROPERTIES ('replication_num' = '1')"; //todo: modify replication num like statistic sys tbl + private static final String BATCH_INSERT_INTO_META_TABLE_TEMPLATE = + "INSERT INTO " + FULL_QUALIFIED_META_TBL_NAME + " values"; + + private static final String INSERT_INTO_META_TABLE_TEMPLATE = + "('${id}', '${job_id}', '${table_name}', '${chunk_list}')"; + + private static final ObjectMapper objectMapper = new ObjectMapper(); + + public static void createMetaTableIfNotExist() throws Exception { + Optional optionalDatabase = + Env.getCurrentEnv().getInternalCatalog() + .getDb(FeConstants.INTERNAL_DB_NAME); + if (!optionalDatabase.isPresent()) { + // should not happen + throw new JobException("Internal database does not exist"); + } + Database database = optionalDatabase.get(); + Table t = database.getTableNullable(FULL_QUALIFIED_META_TBL_NAME); + if (t == null) { + executeInsert(String.format(CREATE_META_TABLE, FULL_QUALIFIED_META_TBL_NAME)); + } + + // double check + t = database.getTableNullable(INTERNAL_STREAMING_JOB_META_TABLE_NAME); + if (t == null){ + throw new JobException(String.format("Table %s doesn't exist", FULL_QUALIFIED_META_TBL_NAME)); + } + } + + public static void insertSplitsToMeta(Long jobId, Map> tableSplits) throws Exception { + List values = new ArrayList<>(); + for (Map.Entry> entry : tableSplits.entrySet()) { + Map params = new HashMap<>(); + params.put("id", UUID.randomUUID().toString().replace("-","")); + params.put("job_id", jobId + ""); + params.put("table_name", entry.getKey()); + params.put("chunk_list", objectMapper.writeValueAsString(entry.getValue())); + StringSubstitutor stringSubstitutor = new StringSubstitutor(params); + String sql = stringSubstitutor.replace(INSERT_INTO_META_TABLE_TEMPLATE); + values.add(sql); + } + batchInsert(values); + } + + private static void batchInsert(List values) throws Exception { + if (values.isEmpty()) { + return; + } + StringBuilder query = new StringBuilder(BATCH_INSERT_INTO_META_TABLE_TEMPLATE); + for (int i = 0; i < values.size(); i++) { + query.append(values.get(i)); + if (i + 1 != values.size()) { + query.append(","); + } else { + query.append(";"); + } + } + executeInsert(query.toString()); + } + + private static void executeInsert(String sql) throws Exception { + try (AutoCloseConnectContext r = buildConnectContext()) { + StmtExecutor stmtExecutor = new StmtExecutor(r.connectContext, sql); + stmtExecutor.execute(); + } + } + + private static AutoCloseConnectContext buildConnectContext() { + ConnectContext connectContext = new ConnectContext(); + connectContext.getState().setInternal(true); + SessionVariable sessionVariable = connectContext.getSessionVariable(); + sessionVariable.setEnableInsertStrict(true); + sessionVariable.setInsertMaxFilterRatio(1); + sessionVariable.enableProfile = false; + connectContext.setEnv(Env.getCurrentEnv()); + connectContext.setDatabase(FeConstants.INTERNAL_DB_NAME); + UUID uuid = UUID.randomUUID(); + TUniqueId queryId = new TUniqueId(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits()); + connectContext.setQueryId(queryId); + connectContext.setStartTime(); + connectContext.setCurrentUserIdentity(UserIdentity.ADMIN); + return new AutoCloseConnectContext(connectContext); + } + + private static JdbcClient getJdbcClient(DataSourceType sourceType, Map properties) throws JobException { + JdbcClientConfig config = new JdbcClientConfig(); + config.setCatalog(sourceType.name()); + config.setUser(properties.get(USER)); + config.setPassword(properties.get(PASSWORD)); + config.setDriverClass(properties.get(DRIVER_CLASS)); + config.setDriverUrl(properties.get(DRIVER_URL)); + config.setJdbcUrl(properties.get(JDBC_URL)); + switch (sourceType) { + case MYSQL: + JdbcClient client = JdbcMySQLClient.createJdbcClient(config); + return client; + default: + throw new JobException("Unsupported source type " + sourceType); + } + } + + public static List generateCreateTableInfos(String targetDb, DataSourceType sourceType, Map properties, Map targetProperties) + throws JobException { + List createtblInfos = new ArrayList<>(); + String includeTables = properties.get(INCLUDE_TABLES); + String excludeTables = properties.get(EXCLUDE_TABLES); + List includeTablesList = new ArrayList<>(); + if (includeTables != null) { + includeTablesList = Arrays.asList(includeTables.split(",")); + } + + String database = properties.get(DATABASE); + JdbcClient jdbcClient = getJdbcClient(sourceType, properties); + List tablesNameList = jdbcClient.getTablesNameList(database); + if (tablesNameList.isEmpty()) { + throw new JobException("No tables found in database " + database); + } + Map tableCreateProperties = getTableCreateProperties(targetProperties); + for (String table : tablesNameList) { + if (!includeTablesList.isEmpty() && !includeTablesList.contains(table)) { + log.info("Skip table {} in database {} as it does not in include_tables {}", table, database, + includeTables); + continue; + } + + if(excludeTables != null && excludeTables.contains(table)) { + log.info("Skip table {} in database {} as it in exclude_tables {}", table, database, + excludeTables); + continue; + } + + List columns = jdbcClient.getColumnsFromJdbc(database, table); + + List primaryKeys = columns.stream().filter(Column::isKey).map(Column::getName) + .collect(Collectors.toList()); + if (primaryKeys.isEmpty()) { + primaryKeys.add(columns.get(0).getName()); + log.info("table {} no primary key, use first column {} to primary key", table, columns.get(0).getName()); + } + // Convert Column to ColumnDefinition + List columnDefinitions = columns.stream().map(col -> { + DataType dataType = DataType.fromCatalogType(col.getType()); + boolean isKey = col.isKey(); + AggregateType aggType = col.getAggregationType(); + boolean isNullable = col.isAllowNull(); + String comment = col.getComment() != null ? col.getComment() : ""; + return new ColumnDefinition(col.getName(), dataType, isKey, aggType, isNullable, + Optional.empty(), comment); + }).collect(Collectors.toList()); + + // Create DistributionDescriptor + DistributionDescriptor distribution = new DistributionDescriptor( + true, // isHash + true, // isAutoBucket + FeConstants.default_bucket_num, + primaryKeys + ); + + // Create CreateTableInfo + CreateTableInfo createtblInfo = new CreateTableInfo( + true, // ifNotExists + false, // isExternal + false, // isTemp + InternalCatalog.INTERNAL_CATALOG_NAME, // ctlName + targetDb, // dbName + table, // tableName + columnDefinitions, // columns + Collections.emptyList(), // indexes + "olap", // engineName + KeysType.UNIQUE_KEYS, // keysType + primaryKeys, // keys + "", // comment + PartitionTableInfo.EMPTY, // partitionTableInfo + distribution, // distribution + Collections.emptyList(), // rollups + tableCreateProperties, // properties + new HashMap<>(), // extProperties + Collections.emptyList() // clusterKeyColumnNames + ); + createtblInfo.analyzeEngine(); + createtblInfos.add(createtblInfo); + } + return createtblInfos; + } + + private static Map getTableCreateProperties(Map properties) { + final Map tableCreateProps = new HashMap<>(); + for (Map.Entry entry : properties.entrySet()) { + if (entry.getKey().startsWith(LoadConstants.TABLE_PROPS_PREFIX)) { + String subKey = entry.getKey().substring(LoadConstants.TABLE_PROPS_PREFIX.length()); + tableCreateProps.put(subKey, entry.getValue()); + } + } + return tableCreateProps; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index f9c3b2e2b2e753..e91e9b191d8294 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -1173,13 +1173,20 @@ public LogicalPlan visitCreateScheduledJob(DorisParser.CreateScheduledJobContext Optional interval = ctx.timeInterval == null ? Optional.empty() : Optional.of(Long.valueOf(ctx.timeInterval.getText())); Optional intervalUnit = ctx.timeUnit == null ? Optional.empty() : Optional.of(ctx.timeUnit.getText()); - Map properties = ctx.propertyClause() != null - ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + Map jobProperties = ctx.jobProperties != null + ? Maps.newHashMap(visitPropertyClause(ctx.jobProperties)) : Maps.newHashMap(); String comment = visitCommentSpec(ctx.commentSpec()); - String executeSql = getOriginSql(ctx.supportedDmlStatement()); + String executeSql = ctx.supportedDmlStatement() == null ? "" : getOriginSql(ctx.supportedDmlStatement()); + Optional sourceType = ctx.sourceType == null ? Optional.empty() : Optional.of(ctx.sourceType.getText()); + String targetDb = ctx.targetDb == null ? "" : ctx.targetDb.getText(); + Map sourceProperties = ctx.sourceProperties != null + ? Maps.newHashMap(visitPropertyItemList(ctx.sourceProperties)) : Maps.newHashMap(); + Map targetProperties = ctx.targetProperties != null + ? Maps.newHashMap(visitPropertyItemList(ctx.targetProperties)) : Maps.newHashMap(); CreateJobInfo createJobInfo = new CreateJobInfo(label, atTime, interval, intervalUnit, startTime, - endsTime, immediateStartOptional, comment, executeSql, ctx.STREAMING() != null, properties); + endsTime, immediateStartOptional, comment, executeSql, ctx.STREAMING() != null, + jobProperties,sourceType, targetDb, sourceProperties, targetProperties); return new CreateJobCommand(createJobInfo); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java index c6ebba5d46167f..c1384efb9dbf2e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.commands.info; +import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; @@ -28,6 +29,7 @@ import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.base.TimerDefinition; +import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.common.IntervalUnit; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.extensions.insert.InsertJob; @@ -70,6 +72,10 @@ public class CreateJobInfo { private final String executeSql; private final boolean streamingJob; private final Map jobProperties; + private final Optional sourceType; + private final String targetDb; + private final Map sourceProperties; + private final Map targetProperties; /** * Constructor for CreateJobInfo. @@ -88,7 +94,8 @@ public CreateJobInfo(Optional labelNameOptional, Optional onceJo Optional intervalOptional, Optional intervalTimeUnitOptional, Optional startsTimeStampOptional, Optional endsTimeStampOptional, Optional immediateStartOptional, String comment, String executeSql, - boolean streamingJob, Map jobProperties) { + boolean streamingJob, Map jobProperties, Optional sourceType, + String targetDb, Map sourceProperties, Map targetProperties) { this.labelNameOptional = labelNameOptional; this.onceJobStartTimestampOptional = onceJobStartTimestampOptional; this.intervalOptional = intervalOptional; @@ -100,6 +107,10 @@ public CreateJobInfo(Optional labelNameOptional, Optional onceJo this.executeSql = executeSql; this.streamingJob = streamingJob; this.jobProperties = jobProperties; + this.sourceType = sourceType; + this.targetDb = targetDb; + this.sourceProperties = sourceProperties; + this.targetProperties = targetProperties; } /** @@ -140,8 +151,22 @@ public AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws UserExcepti } else { buildRecurringJob(timerDefinition, jobExecutionConfiguration); } + jobExecutionConfiguration.setTimerDefinition(timerDefinition); - return analyzeAndCreateJob(executeSql, dbName, jobExecutionConfiguration, jobProperties); + // set source type + if (streamingJob) { + if (sourceType.isPresent()) { + DataSourceType dataSourceType = DataSourceType.valueOf(sourceType.get()); + return analyzeAndCreateFromSourceJob(dbName, jobExecutionConfiguration, jobProperties, targetDb, dataSourceType, sourceProperties, targetProperties); + }else { + return analyzeAndCreateStreamingInsertJob(executeSql, dbName, jobExecutionConfiguration, jobProperties); + } + } else { + if (sourceType.isPresent()) { + throw new AnalysisException("From..To Database is only supported in streaming job"); + } + return analyzeAndCreateInsertJob(executeSql, dbName, jobExecutionConfiguration); + } } private void buildStreamingJob(TimerDefinition timerDefinition) { @@ -216,7 +241,7 @@ private void buildRecurringJob(TimerDefinition timerDefinition, protected void checkAuth() throws AnalysisException { if (streamingJob) { - StreamingInsertJob.checkPrivilege(ConnectContext.get(), executeSql); + StreamingInsertJob.checkPrivilege(ConnectContext.get(), executeSql, targetDb); return; } @@ -226,15 +251,30 @@ protected void checkAuth() throws AnalysisException { } /** - * Analyzes the provided SQL statement and creates an appropriate job based on the parsed logical plan. - * Currently, only "InsertIntoTableCommand" is supported for job creation. + * Analyzes From Source To Database. * - * @param sql the SQL statement to be analyzed - * @param currentDbName the current database name where the SQL statement will be executed - * @param jobExecutionConfiguration the configuration for job execution * @return an instance of AbstractJob corresponding to the SQL statement * @throws UserException if there is an error during SQL analysis or job creation */ + private AbstractJob analyzeAndCreateFromSourceJob(String currentDbName, + JobExecutionConfiguration jobExecutionConfiguration, Map jobProperties, String targetDb, + DataSourceType dataSourceType, Map sourceProperties, Map targetProperties) throws UserException { + Optional db = Env.getCurrentEnv().getInternalCatalog().getDb(targetDb); + if (!db.isPresent()) { + throw new AnalysisException("Target database " + targetDb + " does not exist"); + } + return new StreamingInsertJob(labelNameOptional.get(), + JobStatus.PENDING, + currentDbName, + comment, + ConnectContext.get().getCurrentUserIdentity(), + jobExecutionConfiguration, + System.currentTimeMillis(), + "", + jobProperties, targetDb, dataSourceType, sourceProperties, targetProperties); + } + + private AbstractJob analyzeAndCreateJob(String sql, String currentDbName, JobExecutionConfiguration jobExecutionConfiguration, Map properties) throws UserException { @@ -245,6 +285,16 @@ private AbstractJob analyzeAndCreateJob(String sql, String currentDbName, } } + /** + * Analyzes the provided SQL statement and creates an appropriate job based on the parsed logical plan. + * Currently, only "InsertIntoTableCommand" is supported for job creation. + * + * @param sql the SQL statement to be analyzed + * @param currentDbName the current database name where the SQL statement will be executed + * @param jobExecutionConfiguration the configuration for job execution + * @return an instance of AbstractJob corresponding to the SQL statement + * @throws UserException if there is an error during SQL analysis or job creation + */ private AbstractJob analyzeAndCreateInsertJob(String sql, String currentDbName, JobExecutionConfiguration jobExecutionConfiguration) throws UserException { NereidsParser parser = new NereidsParser(); @@ -270,6 +320,16 @@ private AbstractJob analyzeAndCreateInsertJob(String sql, String currentDbName, } } + /** + * Analyzes the provided SQL statement and creates an appropriate job based on the parsed logical plan. + * Currently, only "InsertIntoTableCommand" is supported for job creation. + * + * @param sql the SQL statement to be analyzed + * @param currentDbName the current database name where the SQL statement will be executed + * @param jobExecutionConfiguration the configuration for job execution + * @return an instance of AbstractJob corresponding to the SQL statement + * @throws UserException if there is an error during SQL analysis or job creation + */ private AbstractJob analyzeAndCreateStreamingInsertJob(String sql, String currentDbName, JobExecutionConfiguration jobExecutionConfiguration, Map properties) throws UserException { NereidsParser parser = new NereidsParser(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index d754952fbee9b9..84764b849aa8ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -67,6 +67,7 @@ import org.apache.doris.datasource.maxcompute.MaxComputeExternalCatalog; import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.job.common.JobType; +import org.apache.doris.job.extensions.insert.streaming.AbstractStreamingTask; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; import org.apache.doris.job.extensions.mtmv.MTMVJob; @@ -1231,8 +1232,8 @@ private static TFetchSchemaTableDataResult taskMetadataResult(TMetadataTableRequ if (job instanceof StreamingInsertJob) { StreamingInsertJob streamingJob = (StreamingInsertJob) job; - List streamingInsertTasks = streamingJob.queryAllStreamTasks(); - for (StreamingInsertTask task : streamingInsertTasks) { + List streamingInsertTasks = streamingJob.queryAllStreamTasks(); + for (AbstractStreamingTask task : streamingInsertTasks) { TRow tvfInfo = task.getTvfInfo(job.getJobName()); if (tvfInfo != null) { dataBatch.add(tvfInfo); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java index b0b2f000dd09cc..beebb7f310fd03 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java @@ -18,13 +18,16 @@ package org.apache.doris.cdcclient.constants; public class LoadConstants { - public static final String DB_SOURCE_TYPE = "db_source_type"; - public static final String HOST = "host"; - public static final String PORT = "port"; - public static final String USERNAME = "username"; + public static final String JDBC_URL = "jdbc_url"; + public static final String DRIVER_URL = "driver_url"; + public static final String DRIVER_CLASS = "driver_class"; + public static final String USER = "user"; public static final String PASSWORD = "password"; public static final String DATABASE = "database"; public static final String INCLUDE_TABLES = "include_tables"; public static final String EXCLUDE_TABLES = "exclude_tables"; + // initial,earliest-offset,latest-offset,specific-offset,timestamp,snapshot + public static final String STARTUP_MODE = "startup_mode"; public static final String SPLIT_SIZE = "split_size"; + public static final String TABLE_PROPS_PREFIX = "table.create.properties."; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index 0671e41941b15b..53661d0f3bc1a2 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -17,13 +17,10 @@ package org.apache.doris.cdcclient.controller; -import java.util.List; import org.apache.doris.cdcclient.common.Env; import org.apache.doris.cdcclient.model.request.FetchRecordReq; import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; import org.apache.doris.cdcclient.model.request.WriteRecordReq; -import org.apache.doris.cdcclient.model.response.RecordWithMeta; -import org.apache.doris.cdcclient.model.response.WriteMetaResp; import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; import org.apache.doris.cdcclient.service.PipelineCoordinator; import org.apache.doris.cdcclient.source.reader.SourceReader; @@ -35,6 +32,7 @@ import org.springframework.web.bind.annotation.RequestMapping; import org.springframework.web.bind.annotation.RequestMethod; import org.springframework.web.bind.annotation.RestController; +import java.util.List; @RestController public class ClientController { @@ -59,8 +57,8 @@ public Object fetchSplits(@RequestBody FetchTableSplitsReq ftsReq) { @RequestMapping(path = "/api/fetchRecords", method = RequestMethod.POST) public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { try { - RecordWithMeta response = pipelineCoordinator.read(recordReq); - return ResponseEntityBuilder.ok(response); + SourceReader reader = Env.getCurrentEnv().getReader(recordReq); + return ResponseEntityBuilder.ok(reader.read(recordReq)); } catch (Exception ex) { LOG.error("Failed fetch record, jobId={}", recordReq.getJobId(), ex); return ResponseEntityBuilder.badRequest(ex.getMessage()); @@ -70,13 +68,15 @@ public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { /** Fetch records from source reader and Write records to backend */ @RequestMapping(path = "/api/writeRecords", method = RequestMethod.POST) public Object writeRecord(@RequestBody WriteRecordReq recordReq) { - try { - WriteMetaResp response = pipelineCoordinator.writeRecords(recordReq); - return ResponseEntityBuilder.ok(response); - } catch (Exception ex) { - LOG.error("Failed to write record, jobId={}", recordReq.getJobId(), ex); - return ResponseEntityBuilder.badRequest(ex.getMessage()); - } + pipelineCoordinator.writeRecordsAsync(recordReq); + return ResponseEntityBuilder.ok("Request accepted, processing asynchronously"); + } + + /** Fetch lastest end meta */ + @RequestMapping(path = "/api/fetchEndOffset", method = RequestMethod.POST) + public Object fetchEndOffset(@RequestBody WriteRecordReq recordReq) { + pipelineCoordinator.writeRecordsAsync(recordReq); + return ResponseEntityBuilder.ok("Request accepted, processing asynchronously"); } @RequestMapping(path = "/api/close/{jobId}", method = RequestMethod.POST) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index c4a875f94756cb..289540c54f8ada 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -19,24 +19,35 @@ import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; +import com.fasterxml.jackson.databind.ObjectMapper; import io.debezium.data.Envelope; +import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import org.apache.commons.collections.CollectionUtils; import org.apache.doris.cdcclient.common.Env; -import org.apache.doris.cdcclient.model.request.FetchRecordReq; import org.apache.doris.cdcclient.model.request.WriteRecordReq; -import org.apache.doris.cdcclient.model.response.RecordWithMeta; import org.apache.doris.cdcclient.model.response.WriteMetaResp; import org.apache.doris.cdcclient.sink.DorisBatchStreamLoad; import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; import org.apache.doris.cdcclient.source.reader.SourceReader; import org.apache.doris.cdcclient.source.reader.SplitReadResult; +import org.apache.doris.cdcclient.utils.HttpUtil; import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; +import org.apache.http.HttpHeaders; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.util.EntityUtils; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; import org.slf4j.Logger; @@ -49,19 +60,98 @@ public class PipelineCoordinator { private static final Logger LOG = LoggerFactory.getLogger(PipelineCoordinator.class); private static final String SPLIT_ID = "splitId"; private static final String PURE_BINLOG_PHASE = "pureBinlogPhase"; - + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); // jobId private final Map batchStreamLoadMap = new ConcurrentHashMap<>(); private final SourceRecordDeserializer> serializer; + private final ThreadPoolExecutor executor; + private static final int MAX_CONCURRENT_TASKS = 10; + private static final int QUEUE_CAPACITY = 128; public PipelineCoordinator() { this.serializer = new DebeziumJsonDeserializer(); + this.executor = + new ThreadPoolExecutor( + MAX_CONCURRENT_TASKS, + MAX_CONCURRENT_TASKS, + 60L, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(QUEUE_CAPACITY), + r -> { + Thread t = + new Thread( + r, "async-write-record-" + System.currentTimeMillis()); + t.setDaemon(false); + return t; + }, + new ThreadPoolExecutor.AbortPolicy()); + } + + public CompletableFuture writeRecordsAsync(WriteRecordReq writeRecordReq) { + return CompletableFuture.runAsync( + () -> { + WriteMetaResp response = null; + Exception error = null; + + try { + LOG.info( + "Start processing async write record, jobId={}", + writeRecordReq.getJobId()); + response = writeRecords(writeRecordReq); + LOG.info( + "Successfully processed async write record, jobId={}", + writeRecordReq.getJobId()); + + } catch (Exception ex) { + LOG.error( + "Failed to process async write record, jobId={}", + writeRecordReq.getJobId(), + ex); + error = ex; + } finally { + // commitTransaction(writeRecordReq.getJobId(), response, error); + } + }, + executor); } - /** Read data from SourceReader and return it with meta information. */ - public RecordWithMeta read(FetchRecordReq recordReq) throws Exception { - SourceReader reader = Env.getCurrentEnv().getReader(recordReq); - return reader.read(recordReq); + /** commit transaction. */ + private void commitTransaction(Long jobId, WriteMetaResp response, Exception error) { + try { + // 序列化为JSON + String jsonBody = OBJECT_MAPPER.writeValueAsString(""); + HttpPost httpPost = new HttpPost("url"); + httpPost.setHeader(HttpHeaders.CONTENT_TYPE, "application/json; charset=UTF-8"); + httpPost.setEntity(new StringEntity(jsonBody, StandardCharsets.UTF_8)); + + LOG.info("Calling callback URL: {}, jobId={}", "url", jobId); + + try (CloseableHttpClient httpClient = HttpUtil.getHttpClient()) { + try (CloseableHttpResponse httpResponse = httpClient.execute(httpPost)) { + int statusCode = httpResponse.getStatusLine().getStatusCode(); + String responseBody = + httpResponse.getEntity() != null + ? EntityUtils.toString(httpResponse.getEntity()) + : ""; + + LOG.info( + "Callback completed, jobId={}, statusCode={}, response={}", + jobId, + statusCode, + responseBody); + + if (statusCode < 200 || statusCode >= 300) { + LOG.warn( + "Callback returned non-2xx status, jobId={}, statusCode={}", + jobId, + statusCode); + } + } + } + + } catch (Exception ex) { + LOG.error("Failed to call callback URL: {}, jobId={}", "", jobId, ex); + } } /** Read data from SourceReader and write it to Doris, while returning meta information. */ diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index 9d3e1acf5d9205..3634f5086a6655 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -17,6 +17,7 @@ package org.apache.doris.cdcclient.utils; +import com.mysql.cj.conf.ConnectionUrl; import java.util.Map; import java.util.Properties; import org.apache.commons.lang3.StringUtils; @@ -34,11 +35,13 @@ public class ConfigUtil { public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { Map cdcConfig = config.getConfig(); MySqlSourceConfigFactory configFactory = new MySqlSourceConfigFactory(); - configFactory.hostname(cdcConfig.get(LoadConstants.HOST)); - configFactory.port(Integer.valueOf(cdcConfig.get(LoadConstants.PORT))); - configFactory.username(cdcConfig.get(LoadConstants.USERNAME)); - configFactory.password(cdcConfig.get(LoadConstants.PASSWORD)); + ConnectionUrl cu = + ConnectionUrl.getConnectionUrlInstance(cdcConfig.get(LoadConstants.JDBC_URL), null); + configFactory.hostname(cu.getMainHost().getHost()); + configFactory.port(cu.getMainHost().getPort()); + configFactory.username(cdcConfig.get(LoadConstants.USER)); + configFactory.password(cdcConfig.get(LoadConstants.PASSWORD)); String databaseName = cdcConfig.get(LoadConstants.DATABASE); configFactory.databaseList(databaseName); configFactory.serverId(String.valueOf(Math.abs(config.getJobId().hashCode()))); @@ -107,8 +110,7 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { } Properties jdbcProperteis = new Properties(); - jdbcProperteis.put("allowPublicKeyRetrieval", "true"); - jdbcProperteis.put("useSSL", "false"); + jdbcProperteis.putAll(cu.getOriginalProperties()); configFactory.jdbcProperties(jdbcProperteis); if (cdcConfig.containsKey(LoadConstants.SPLIT_SIZE)) { From 38507ae87db40522a64b84d05e5e635cd0142c19 Mon Sep 17 00:00:00 2001 From: wudi Date: Thu, 4 Dec 2025 20:31:53 +0800 Subject: [PATCH 08/27] add fe create streamt multi ask --- .../streaming/AbstractStreamingTask.java | 53 ++++- .../insert/streaming/StreamingInsertJob.java | 45 +++- .../insert/streaming/StreamingInsertTask.java | 42 +--- .../streaming/StreamingJobSchedulerTask.java | 9 +- .../streaming/StreamingMultiTblTask.java | 166 ++++++++++++- .../doris/job/offset/jdbc/JdbcOffset.java | 2 +- .../offset/jdbc/JdbcSourceOffsetProvider.java | 79 ++++++- .../jdbc/split/AbstractSourceSplit.java | 8 +- .../job/offset/jdbc/split/BinlogSplit.java | 17 +- .../job/offset/jdbc/split/SnapshotSplit.java | 8 + .../config/GlobalExceptionHandler.java | 24 ++ .../cdcclient/constants/LoadConstants.java | 2 + .../controller/ClientController.java | 16 +- .../model/request/JobBaseRecordReq.java | 2 +- .../model/request/WriteRecordReq.java | 3 +- .../service/PipelineCoordinator.java | 220 +++++++----------- .../cdcclient/sink/DorisBatchStreamLoad.java | 134 ++++++++--- .../doris/cdcclient/sink/HttpPutBuilder.java | 135 +++++++++++ .../deserialize/DebeziumJsonDeserializer.java | 4 +- .../cdcclient/source/reader/SourceReader.java | 3 + .../reader/mysql/MySqlSourceReader.java | 81 ++++--- .../source/split/AbstractSourceSplit.java | 7 +- .../cdcclient/source/split/BinlogSplit.java | 13 +- 23 files changed, 786 insertions(+), 287 deletions(-) create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java index 1f8589f897ce63..393f2e3df6e835 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java @@ -3,13 +3,16 @@ import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.Env; import org.apache.doris.common.FeConstants; +import org.apache.doris.common.Status; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.job.base.Job; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.offset.Offset; import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; +import org.apache.doris.thrift.TStatusCode; import lombok.Getter; import lombok.Setter; @@ -24,12 +27,14 @@ @Getter public abstract class AbstractStreamingTask { private static final int MAX_RETRY = 3; + private static final String LABEL_SPLITTER = "_"; private int retryCount = 0; protected String labelName; protected Offset runningOffset; protected UserIdentity userIdentity; @Setter protected volatile TaskStatus status; + @Setter protected String errMsg; protected long jobId; protected long taskId; @@ -39,14 +44,18 @@ public abstract class AbstractStreamingTask { @Getter private AtomicBoolean isCanceled = new AtomicBoolean(false); + public AbstractStreamingTask(long jobId, long taskId) { + this.jobId = jobId; + this.taskId = taskId; + this.labelName = getJobId() + LABEL_SPLITTER + getTaskId(); + this.createTimeMs = System.currentTimeMillis(); + } + public abstract void before() throws Exception; public abstract void run() throws JobException ; public abstract boolean onSuccess() throws JobException; - public abstract void onFail(String errMsg) throws JobException; - public abstract void cancel(boolean needWaitCancelComplete); public abstract void closeOrReleaseResources(); - public void execute() throws JobException { while (retryCount <= MAX_RETRY) { try { @@ -78,6 +87,44 @@ public void execute() throws JobException { } } + protected void onFail(String errMsg) throws JobException { + if (getIsCanceled().get()) { + return; + } + this.errMsg = errMsg; + this.status = TaskStatus.FAILED; + this.finishTimeMs = System.currentTimeMillis(); + if (!isCallable()) { + return; + } + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + StreamingInsertJob streamingInsertJob = (StreamingInsertJob) job; + streamingInsertJob.onStreamTaskFail(this); + } + + protected boolean isCallable() { + if (status.equals(TaskStatus.CANCELED)) { + return false; + } + if (null != Env.getCurrentEnv().getJobManager().getJob(jobId)) { + return true; + } + return false; + } + + public void cancel(boolean needWaitCancelComplete) { + if (TaskStatus.SUCCESS.equals(status) || TaskStatus.FAILED.equals(status) + || TaskStatus.CANCELED.equals(status)) { + return; + } + status = TaskStatus.CANCELED; + if (getIsCanceled().get()) { + return; + } + getIsCanceled().getAndSet(true); + this.errMsg = "task cancelled"; + } + /** * show streaming insert task info detail */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 28d8fa5ec965ed..a87ff6466c61dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -52,6 +52,7 @@ import org.apache.doris.job.util.StreamingJobUtils; import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.loadv2.LoadStatistic; +import org.apache.doris.load.routineload.RoutineLoadTaskInfo; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundTVFRelation; @@ -417,11 +418,10 @@ public List createTasks(TaskType taskType, Map originTvfProps, UserIdentity userIdentity) { - this.jobId = jobId; - this.taskId = taskId; + super(jobId, taskId); this.sql = sql; this.userIdentity = userIdentity; this.currentDb = currentDb; this.offsetProvider = offsetProvider; this.jobProperties = jobProperties; this.originTvfProps = originTvfProps; - this.labelName = getJobId() + LABEL_SPLITTER + getTaskId(); - this.createTimeMs = System.currentTimeMillis(); } @Override @@ -158,33 +154,13 @@ public boolean onSuccess() throws JobException { } @Override - public void onFail(String errMsg) throws JobException { - if (getIsCanceled().get()) { - return; - } - this.errMsg = errMsg; - this.status = TaskStatus.FAILED; - this.finishTimeMs = System.currentTimeMillis(); - if (!isCallable()) { - return; - } - Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); - StreamingInsertJob streamingInsertJob = (StreamingInsertJob) job; - streamingInsertJob.onStreamTaskFail(this); + protected void onFail(String errMsg) throws JobException { + super.onFail(errMsg); } @Override public void cancel(boolean needWaitCancelComplete) { - if (TaskStatus.SUCCESS.equals(status) || TaskStatus.FAILED.equals(status) - || TaskStatus.CANCELED.equals(status)) { - return; - } - status = TaskStatus.CANCELED; - if (getIsCanceled().get()) { - return; - } - getIsCanceled().getAndSet(true); - this.errMsg = "task cancelled"; + super.cancel(needWaitCancelComplete); if (null != stmtExecutor) { log.info("cancelling streaming insert task, job id is {}, task id is {}", getJobId(), getTaskId()); @@ -205,14 +181,4 @@ public void closeOrReleaseResources() { ctx = null; } } - - private boolean isCallable() { - if (status.equals(TaskStatus.CANCELED)) { - return false; - } - if (null != Env.getCurrentEnv().getJobManager().getJob(jobId)) { - return true; - } - return false; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java index 20d030534d586a..b549195211cd36 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -44,7 +44,7 @@ public void run() throws JobException { handlePendingState(); break; case RUNNING: - streamingInsertJob.fetchMeta(); + handleRunningState(); break; case PAUSED: autoResumeHandler(); @@ -66,11 +66,16 @@ private void handlePendingState() throws JobException { } } - streamingInsertJob.createStreamingTask();; + streamingInsertJob.createStreamingTask(); streamingInsertJob.updateJobStatus(JobStatus.RUNNING); streamingInsertJob.setAutoResumeCount(0); } + private void handleRunningState() { + streamingInsertJob.processTimeoutTasks(); + streamingInsertJob.fetchMeta(); + } + private void autoResumeHandler() throws JobException { final FailureReason failureReason = streamingInsertJob.getFailureReason(); final long latestAutoResumeTimestamp = streamingInsertJob.getLatestAutoResumeTimestamp(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java index 089c3553b4f3b5..76f7f9fa551de1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java @@ -1,27 +1,54 @@ package org.apache.doris.job.extensions.insert.streaming; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Status; +import org.apache.doris.httpv2.entity.ResponseBody; +import org.apache.doris.httpv2.rest.RestApiStatusCode; +import org.apache.doris.job.base.Job; +import org.apache.doris.job.common.DataSourceType; +import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.job.offset.jdbc.JdbcOffset; +import org.apache.doris.job.offset.jdbc.split.SnapshotSplit; +import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.thrift.TStatusCode; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import lombok.Getter; import lombok.extern.log4j.Log4j2; +import org.apache.http.HttpHeaders; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.util.EntityUtils; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; import java.util.Map; @Log4j2 @Getter public class StreamingMultiTblTask extends AbstractStreamingTask { - - SourceOffsetProvider offsetProvider; - Map sourceProperties; - Map targetProperties; + private static final ObjectMapper objectMapper = new ObjectMapper(); + private DataSourceType dataSourceType; + private SourceOffsetProvider offsetProvider; + private Map sourceProperties; + private Map targetProperties; + private String targetDb; public StreamingMultiTblTask(Long jobId, long taskId, + DataSourceType dataSourceType, SourceOffsetProvider offsetProvider, Map sourceProperties, + String targetDb, Map targetProperties) { - this.jobId = jobId; - this.taskId = taskId; + super(jobId, taskId); + this.dataSourceType = dataSourceType; this.offsetProvider = offsetProvider; this.sourceProperties = sourceProperties; this.targetProperties = targetProperties; @@ -29,33 +56,148 @@ public StreamingMultiTblTask(Long jobId, @Override public void before() throws Exception { - log.info("StreamingMultiTblTask before execution."); + if (getIsCanceled().get()) { + log.info("streaming multi task has been canceled, task id is {}", getTaskId()); + return; + } + this.status = TaskStatus.RUNNING; + this.startTimeMs = System.currentTimeMillis(); this.runningOffset = offsetProvider.getNextOffset(null, sourceProperties); + log.info("streaming multi task {} get running offset: {}", taskId, runningOffset.toString()); } @Override public void run() throws JobException { - log.info("StreamingMultiTblTask run execution."); + if (getIsCanceled().get()) { + log.info("task has been canceled, task id is {}", getTaskId()); + return; + } + log.info("start to run streaming multi task, offset is {}", runningOffset.toString()); + sendWriteRequest(); + } + + private void sendWriteRequest() throws JobException { + Map params = buildRequestParams(); + + String url = "http://127.0.0.1:9096/api/writeRecords"; + // Prepare request body + String requestBody = null; + try { + requestBody = objectMapper.writeValueAsString(params); + }catch (IOException e) { + throw new JobException("Failed to serialize request body: " + e.getMessage(), e); + } + // Create HTTP POST request + HttpPost httpPost = new HttpPost(url); + httpPost.addHeader("token", getToken()); + StringEntity stringEntity = new StringEntity(requestBody, "UTF-8"); + stringEntity.setContentType("application/json"); + httpPost.setEntity(stringEntity); + // Set request headers + httpPost.setHeader(HttpHeaders.CONTENT_TYPE, "application/json"); + + // Execute request + try (CloseableHttpClient client = HttpClientBuilder.create().build()) { + String response = client.execute(httpPost, httpResponse -> { + int statusCode = httpResponse.getStatusLine().getStatusCode(); + String responseBody = EntityUtils.toString(httpResponse.getEntity(), "UTF-8"); + if (statusCode != 200) { + throw new RuntimeException("Failed to get split from CDC client, HTTP status code: " + statusCode); + } + return responseBody; + }); + + ResponseBody responseObj = objectMapper.readValue( + response, + new TypeReference>() {} + ); + if (responseObj.getCode() == RestApiStatusCode.OK.code) { + log.info("Send write records request successfully, response: {}", responseObj.getData()); + return; + } + throw new JobException("Failed to send write records request , error message: " + responseObj); + } catch (Exception ex) { + log.error("Send write request: ", ex); + throw new JobException("Failed to send write request: " + ex.getMessage(), ex); + } + + /** + Backend backend = selectBackend(jobId); + Map params = buildSplitParams(table); + InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() + .setApi("/api/fetchSplits") + .setParams(GsonUtils.GSON.toJson(params)).build(); + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); + InternalService.PRequestCdcClientResult result = null; + try { + Future future = + BackendServiceProxy.getInstance().requestCdcClient(address, request); + result = future.get(); + TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + log.error("Failed to get split from backend, {}", result.getStatus().getErrorMsgs(0)); + throw new JobException("Failed to get split from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + result.getResponse()); + } + } catch (ExecutionException | InterruptedException ex) { + log.error("Get splits error: ", ex); + throw new JobException(ex); + } + log.info("========fetch cdc split {}", result.getResponse()); + return ""; + **/ + } + + private String getToken() throws JobException { + String token = ""; + try { + // Acquire token from master + token = Env.getCurrentEnv().getTokenManager().acquireToken(); + } catch (Exception e) { + log.warn("Failed to get auth token:", e); + throw new JobException(e.getMessage()); + } + return token; + } + + private Map buildRequestParams() { + JdbcOffset offset = (JdbcOffset) runningOffset; + Map params = new HashMap<>(); + params.put("jobId", getJobId()); + params.put("labelName", getLabelName()); + params.put("dataSource", dataSourceType); + params.put("meta", offset.getSplit()); + params.put("config", sourceProperties); + params.put("targetDb", targetDb); + return params; } @Override public boolean onSuccess() throws JobException { - log.info("StreamingMultiTblTask onSuccess execution."); + if (getIsCanceled().get()) { + return false; + } + log.info("streaming multi task {} send write request run successfully.", getTaskId()); return false; } @Override - public void onFail(String errMsg) throws JobException { - + protected void onFail(String errMsg) throws JobException { + super.onFail(errMsg); } @Override public void cancel(boolean needWaitCancelComplete) { - + // No manual cancellation is required; the task ID will be checked for consistency in the beforeCommit function. + super.cancel(needWaitCancelComplete); } @Override public void closeOrReleaseResources() { + // close cdc client connection + } + public boolean isTimeout() { + //todo: need to config + return (System.currentTimeMillis() - createTimeMs) > 300 * 1000; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java index 6545a1969bdd66..160aa9e1d46915 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -10,7 +10,7 @@ @Setter public class JdbcOffset implements Offset { - private Map meta; + private AbstractSourceSplit split; @Override public String toSerializedJson() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index 6a6e87b1626d3a..7717039ee173a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -7,6 +7,7 @@ import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.job.offset.jdbc.split.BinlogSplit; import org.apache.doris.job.offset.jdbc.split.SnapshotSplit; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.persist.gson.GsonUtils; @@ -37,13 +38,14 @@ @Setter @Log4j2 public class JdbcSourceOffsetProvider implements SourceOffsetProvider { + public static final String BINLOG_SPLIT_ID = "binlog-split"; private static final ObjectMapper objectMapper = new ObjectMapper(); private Long jobId; private DataSourceType sourceType; private Map sourceProperties = new HashMap<>(); List remainingSplits; - List assignedSplits; + List finishedSplits; JdbcOffset currentOffset; Map endBinlogOffset; @@ -55,10 +57,23 @@ public String getSourceType() { @Override public Offset getNextOffset(StreamingJobProperties jobProps, Map properties) { - // 全量从系统表中获取 - - // 增量直接就是上一次的offset - return null; + JdbcOffset nextOffset = new JdbcOffset(); + if (!remainingSplits.isEmpty()) { + // snapshot read + SnapshotSplit snapshotSplit = remainingSplits.get(0); + nextOffset.setSplit(snapshotSplit); + return nextOffset; + }else if (currentOffset.getSplit().snapshotSplit()){ + // snapshot to binlog + BinlogSplit binlogSplit = new BinlogSplit(); + binlogSplit.setSplitId(BINLOG_SPLIT_ID); + binlogSplit.setFinishedSplits(finishedSplits); + nextOffset.setSplit(binlogSplit); + return nextOffset; + } else { + // only binlog + return currentOffset; + } } @Override @@ -84,12 +99,53 @@ public void updateOffset(Offset offset) { @Override public void fetchRemoteMeta(Map properties) throws Exception { - // todo: request cdc client api + // todo: change to request be + Map params = buildBaseParams(); + String url = "http://127.0.0.1:9096/api/fetchEndOffset"; + // Prepare request body + String requestBody = GsonUtils.GSON.toJson(params); + + // Create HTTP POST request + HttpPost httpPost = new HttpPost(url); + StringEntity stringEntity = new StringEntity(requestBody, "UTF-8"); + stringEntity.setContentType("application/json"); + httpPost.setEntity(stringEntity); + // Set request headers + httpPost.setHeader("Content-Type", "application/json"); + + // Execute request + try (CloseableHttpClient client = HttpClientBuilder.create().build()) { + String response = client.execute(httpPost, httpResponse -> { + int statusCode = httpResponse.getStatusLine().getStatusCode(); + String responseBody = EntityUtils.toString(httpResponse.getEntity(), "UTF-8"); + if (statusCode != 200) { + throw new RuntimeException("Failed to get remote offset from CDC client, HTTP status code: " + statusCode); + } + return responseBody; + }); + + ResponseBody> responseObj = objectMapper.readValue( + response, + new TypeReference>>() {} + ); + endBinlogOffset = responseObj.getData(); + } catch (Exception ex) { + log.error("Get splits error: ", ex); + throw new JobException("Failed to request CDC client: " + ex.getMessage(), ex); + } } @Override public boolean hasMoreDataToConsume() { - return false; + if(!remainingSplits.isEmpty()) { + return true; + } + if (currentOffset != null && !currentOffset.getSplit().snapshotSplit()) { + BinlogSplit binlogSplit = (BinlogSplit) currentOffset.getSplit(); + Map startingOffset = binlogSplit.getStartingOffset(); + // todo: should compare offset + } + return true; } @Override @@ -191,15 +247,20 @@ private List requestTableSplits(String table) throws JobException **/ } - private Map buildSplitParams(String table) { + private Map buildBaseParams() { Map params = new HashMap<>(); params.put("jobId", getJobId()); params.put("dataSource", sourceType); - params.put("snapshotTable", table); params.put("config", sourceProperties); return params; } + private Map buildSplitParams(String table) { + Map params = buildBaseParams(); + params.put("snapshotTable", table); + return params; + } + public static Backend selectBackend(Long jobId) throws JobException { Backend backend = null; BeSelectionPolicy policy = null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java index 16e653fce9beef..c977876030662e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java @@ -17,16 +17,22 @@ package org.apache.doris.job.offset.jdbc.split; +import com.fasterxml.jackson.databind.ObjectMapper; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.Setter; +import java.io.Serializable; @Getter @Setter @AllArgsConstructor @NoArgsConstructor -public abstract class AbstractSourceSplit implements SourceSplit { +public abstract class AbstractSourceSplit implements SourceSplit, Serializable { private static final long serialVersionUID = 1L; protected String splitId; + + public boolean snapshotSplit() { + return this instanceof SnapshotSplit; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java index fff821805ede53..94e4e6e57ab91e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java @@ -17,20 +17,29 @@ package org.apache.doris.job.offset.jdbc.split; +import org.apache.doris.persist.gson.GsonUtils; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; import lombok.EqualsAndHashCode; import lombok.Getter; +import lombok.NoArgsConstructor; import lombok.Setter; +import java.util.List; import java.util.Map; @Getter @Setter @EqualsAndHashCode(callSuper = true) +@NoArgsConstructor public class BinlogSplit extends AbstractSourceSplit { private static final long serialVersionUID = 1L; - private Map offset; + private Map startingOffset; + private Map endingOffset; + // binlog split meta, first binlog split requires + private List finishedSplits; - public BinlogSplit(String splitId, Map offset) { - super(splitId); - this.offset = offset; + @Override + public String toString() { + return new Gson().toJson(this); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java index b037c47f4c5b65..271be64e73a26f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java @@ -17,14 +17,17 @@ package org.apache.doris.job.offset.jdbc.split; +import org.apache.doris.persist.gson.GsonUtils; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; +import com.google.gson.Gson; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.Setter; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Optional; @@ -100,4 +103,9 @@ public static SnapshotSplit fromMap(Map map) throws JsonProcessi public static String getOrEmptyArray(Map map, String key) { return Optional.ofNullable(map.get(key)).orElse("[]"); } + + @Override + public String toString() { + return new Gson().toJson(this); + } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java new file mode 100644 index 00000000000000..145f83969359eb --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java @@ -0,0 +1,24 @@ +package org.apache.doris.cdcclient.config; + +import jakarta.servlet.http.HttpServletRequest; +import lombok.extern.slf4j.Slf4j; +import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.context.MessageSource; +import org.springframework.web.bind.annotation.ControllerAdvice; +import org.springframework.web.bind.annotation.ExceptionHandler; +import org.springframework.web.bind.annotation.ResponseBody; + +@Slf4j +@ControllerAdvice +public class GlobalExceptionHandler { + + @Autowired private MessageSource messageSource; + + @ExceptionHandler(Exception.class) + @ResponseBody + public Object exceptionHandler(HttpServletRequest request, Exception e) { + log.error("Unexpected exception", e); + return ResponseEntityBuilder.internalError(e.getMessage()); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java index beebb7f310fd03..10f3e0b838051e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java @@ -30,4 +30,6 @@ public class LoadConstants { public static final String STARTUP_MODE = "startup_mode"; public static final String SPLIT_SIZE = "split_size"; public static final String TABLE_PROPS_PREFIX = "table.create.properties."; + + public static final String DELETE_SIGN_KEY = "__DORIS_DELETE_SIGN__"; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index 53661d0f3bc1a2..b1f7754655fcf5 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -18,12 +18,14 @@ package org.apache.doris.cdcclient.controller; import org.apache.doris.cdcclient.common.Env; +import org.apache.doris.cdcclient.model.JobConfig; import org.apache.doris.cdcclient.model.request.FetchRecordReq; import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; import org.apache.doris.cdcclient.model.request.WriteRecordReq; import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; import org.apache.doris.cdcclient.service.PipelineCoordinator; import org.apache.doris.cdcclient.source.reader.SourceReader; +import jakarta.servlet.http.HttpServletRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -67,16 +69,20 @@ public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { /** Fetch records from source reader and Write records to backend */ @RequestMapping(path = "/api/writeRecords", method = RequestMethod.POST) - public Object writeRecord(@RequestBody WriteRecordReq recordReq) { - pipelineCoordinator.writeRecordsAsync(recordReq); + public Object writeRecord(@RequestBody WriteRecordReq recordReq, HttpServletRequest request) { + pipelineCoordinator.writeRecordsAsync(recordReq, parseToken(request)); return ResponseEntityBuilder.ok("Request accepted, processing asynchronously"); } + private String parseToken(HttpServletRequest request) { + return request.getHeader("token"); + } + /** Fetch lastest end meta */ @RequestMapping(path = "/api/fetchEndOffset", method = RequestMethod.POST) - public Object fetchEndOffset(@RequestBody WriteRecordReq recordReq) { - pipelineCoordinator.writeRecordsAsync(recordReq); - return ResponseEntityBuilder.ok("Request accepted, processing asynchronously"); + public Object fetchEndOffset(@RequestBody JobConfig jobConfig) { + SourceReader reader = Env.getCurrentEnv().getReader(jobConfig); + return ResponseEntityBuilder.ok(reader.getEndOffset(jobConfig)); } @RequestMapping(path = "/api/close/{jobId}", method = RequestMethod.POST) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java index 6696fe0bcae2a3..dced17b41fb9bb 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java @@ -27,7 +27,7 @@ @Setter @EqualsAndHashCode(callSuper = true) public abstract class JobBaseRecordReq extends JobConfig { - protected Map meta; + protected Map meta; public abstract boolean isReload(); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java index 7550876da12b92..0b92444095affb 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java @@ -24,7 +24,8 @@ @EqualsAndHashCode(callSuper = true) public class WriteRecordReq extends JobBaseRecordReq { private long maxInterval; - private String targetDatabase; + private String targetDb; + private String labelName; @Override public boolean isReload() { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 289540c54f8ada..1113bf1aa43473 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -17,11 +17,9 @@ package org.apache.doris.cdcclient.service; -import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; - import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import io.debezium.data.Envelope; -import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -33,21 +31,14 @@ import java.util.concurrent.TimeUnit; import org.apache.commons.collections.CollectionUtils; import org.apache.doris.cdcclient.common.Env; +import org.apache.doris.cdcclient.exception.StreamLoadException; import org.apache.doris.cdcclient.model.request.WriteRecordReq; -import org.apache.doris.cdcclient.model.response.WriteMetaResp; import org.apache.doris.cdcclient.sink.DorisBatchStreamLoad; import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; import org.apache.doris.cdcclient.source.reader.SourceReader; import org.apache.doris.cdcclient.source.reader.SplitReadResult; -import org.apache.doris.cdcclient.utils.HttpUtil; import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; -import org.apache.http.HttpHeaders; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpPost; -import org.apache.http.entity.StringEntity; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.util.EntityUtils; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; import org.slf4j.Logger; @@ -87,164 +78,117 @@ public PipelineCoordinator() { new ThreadPoolExecutor.AbortPolicy()); } - public CompletableFuture writeRecordsAsync(WriteRecordReq writeRecordReq) { + public CompletableFuture writeRecordsAsync(WriteRecordReq writeRecordReq, String token) { + Preconditions.checkNotNull(token, "doris token must not be null"); + Preconditions.checkNotNull(writeRecordReq.getLabelName(), "labelName must not be null"); + Preconditions.checkNotNull(writeRecordReq.getTargetDb(), "targetDb must not be null"); return CompletableFuture.runAsync( () -> { - WriteMetaResp response = null; - Exception error = null; - try { LOG.info( - "Start processing async write record, jobId={}", - writeRecordReq.getJobId()); - response = writeRecords(writeRecordReq); + "Start processing async write record, labelName={}", + writeRecordReq.getLabelName()); + writeRecords(writeRecordReq, token); LOG.info( - "Successfully processed async write record, jobId={}", - writeRecordReq.getJobId()); - + "Successfully processed async write record, labelName={}", + writeRecordReq.getLabelName()); } catch (Exception ex) { LOG.error( - "Failed to process async write record, jobId={}", - writeRecordReq.getJobId(), + "Failed to process async write record, labelName={}", + writeRecordReq.getLabelName(), ex); - error = ex; - } finally { - // commitTransaction(writeRecordReq.getJobId(), response, error); } }, executor); } - /** commit transaction. */ - private void commitTransaction(Long jobId, WriteMetaResp response, Exception error) { - try { - // 序列化为JSON - String jsonBody = OBJECT_MAPPER.writeValueAsString(""); - HttpPost httpPost = new HttpPost("url"); - httpPost.setHeader(HttpHeaders.CONTENT_TYPE, "application/json; charset=UTF-8"); - httpPost.setEntity(new StringEntity(jsonBody, StandardCharsets.UTF_8)); - - LOG.info("Calling callback URL: {}, jobId={}", "url", jobId); - - try (CloseableHttpClient httpClient = HttpUtil.getHttpClient()) { - try (CloseableHttpResponse httpResponse = httpClient.execute(httpPost)) { - int statusCode = httpResponse.getStatusLine().getStatusCode(); - String responseBody = - httpResponse.getEntity() != null - ? EntityUtils.toString(httpResponse.getEntity()) - : ""; - - LOG.info( - "Callback completed, jobId={}, statusCode={}, response={}", - jobId, - statusCode, - responseBody); - - if (statusCode < 200 || statusCode >= 300) { - LOG.warn( - "Callback returned non-2xx status, jobId={}, statusCode={}", - jobId, - statusCode); - } - } - } - - } catch (Exception ex) { - LOG.error("Failed to call callback URL: {}, jobId={}", "", jobId, ex); - } - } - /** Read data from SourceReader and write it to Doris, while returning meta information. */ - public WriteMetaResp writeRecords(WriteRecordReq writeRecordReq) throws Exception { + public void writeRecords(WriteRecordReq writeRecordReq, String token) throws Exception { SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordReq); - WriteMetaResp recordResponse = new WriteMetaResp(); - SplitReadResult readResult = sourceReader.readSplitRecords(writeRecordReq); - - DorisBatchStreamLoad batchStreamLoad = - getOrCreateBatchStreamLoad(writeRecordReq.getJobId()); - boolean readBinlog = readResult.isReadBinlog(); - boolean pureBinlogPhase = readResult.isPureBinlogPhase(); - - boolean hasData = false; - // Record start time for maxInterval check - long startTime = System.currentTimeMillis(); - long maxIntervalMillis = writeRecordReq.getMaxInterval() * 1000; - - // Use iterators to read and write. - Iterator iterator = readResult.getRecordIterator(); - while (iterator != null && iterator.hasNext()) { - SourceRecord element = iterator.next(); - if (RecordUtils.isDataChangeRecord(element)) { - List serializedRecords = - serializer.deserialize(writeRecordReq.getConfig(), element); - if (!CollectionUtils.isEmpty(serializedRecords)) { - String database = writeRecordReq.getTargetDatabase(); - String table = extractTable(element); - hasData = true; - for (String record : serializedRecords) { - batchStreamLoad.writeRecord(database, table, record.getBytes()); - } - - Map lastMeta = - RecordUtils.getBinlogPosition(element).getOffset(); - if (readBinlog && sourceReader.getSplitId(readResult.getSplit()) != null) { - lastMeta.put(SPLIT_ID, sourceReader.getSplitId(readResult.getSplit())); - lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + DorisBatchStreamLoad batchStreamLoad = null; + Map metaResponse = new HashMap<>(); + try { + SplitReadResult readResult = sourceReader.readSplitRecords(writeRecordReq); + batchStreamLoad = + getOrCreateBatchStreamLoad( + writeRecordReq.getJobId(), writeRecordReq.getTargetDb()); + batchStreamLoad.setCurrentLabel(writeRecordReq.getLabelName()); + batchStreamLoad.setToken(token); + boolean readBinlog = readResult.isReadBinlog(); + boolean pureBinlogPhase = readResult.isPureBinlogPhase(); + + boolean hasData = false; + // Record start time for maxInterval check + long startTime = System.currentTimeMillis(); + long maxIntervalMillis = writeRecordReq.getMaxInterval() * 1000; + + // Use iterators to read and write. + Iterator iterator = readResult.getRecordIterator(); + while (iterator != null && iterator.hasNext()) { + SourceRecord element = iterator.next(); + if (RecordUtils.isDataChangeRecord(element)) { + List serializedRecords = + serializer.deserialize(writeRecordReq.getConfig(), element); + if (!CollectionUtils.isEmpty(serializedRecords)) { + String database = writeRecordReq.getTargetDb(); + String table = extractTable(element); + hasData = true; + for (String record : serializedRecords) { + batchStreamLoad.writeRecord(database, table, record.getBytes()); + } + + Map lastMeta = + RecordUtils.getBinlogPosition(element).getOffset(); + if (readBinlog && sourceReader.getSplitId(readResult.getSplit()) != null) { + lastMeta.put(SPLIT_ID, sourceReader.getSplitId(readResult.getSplit())); + lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + } + metaResponse = lastMeta; } - recordResponse.setMeta(lastMeta); + } + // Check if maxInterval has been exceeded + long elapsedTime = System.currentTimeMillis() - startTime; + if (maxIntervalMillis > 0 && elapsedTime >= maxIntervalMillis) { + LOG.info( + "Max interval {} seconds reached, stopping data reading", + writeRecordReq.getMaxInterval()); + break; } } - // Check if maxInterval has been exceeded - long elapsedTime = System.currentTimeMillis() - startTime; - if (maxIntervalMillis > 0 && elapsedTime >= maxIntervalMillis) { - LOG.info( - "Max interval {} seconds reached, stopping data reading", - writeRecordReq.getMaxInterval()); - break; + if (!hasData) { + // should not happen, No data will be distributed task + throw new StreamLoadException("No data to write"); } - } - if (hasData) { + // wait all stream load finish batchStreamLoad.forceFlush(); - } - - sourceReader.finishSplitRecords(); - // update offset meta - if (!readBinlog) { - Map offsetRes = - sourceReader.extractSnapshotOffset( - readResult.getSplitState(), readResult.getSplit()); - if (offsetRes != null) { - recordResponse.setMeta(offsetRes); - } - } - - if (!hasData) { - if (readBinlog) { + // update offset meta + if (!readBinlog) { Map offsetRes = - sourceReader.extractBinlogOffset(readResult.getSplit(), pureBinlogPhase); - if (offsetRes != null) { - recordResponse.setMeta(offsetRes); - } else { - // Fallback to request meta if extraction fails - Map fallbackOffset = new HashMap<>(writeRecordReq.getMeta()); - fallbackOffset.put(SPLIT_ID, BINLOG_SPLIT_ID); - fallbackOffset.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); - recordResponse.setMeta(fallbackOffset); + sourceReader.extractSnapshotOffset( + readResult.getSplitState(), readResult.getSplit()); + if (offsetRes == null) { + // should not happen + throw new StreamLoadException( + "Chunk data cannot be obtained from highWatermark."); } - } else { - recordResponse.setMeta(writeRecordReq.getMeta()); + metaResponse = offsetRes; + } + batchStreamLoad.commitTransaction(metaResponse); + } finally { + sourceReader.finishSplitRecords(); + if (batchStreamLoad != null) { + batchStreamLoad.resetLabel(); } } - return recordResponse; } - private DorisBatchStreamLoad getOrCreateBatchStreamLoad(Long jobId) { + private DorisBatchStreamLoad getOrCreateBatchStreamLoad(Long jobId, String targetDb) { return batchStreamLoadMap.computeIfAbsent( jobId, k -> { LOG.info("Create DorisBatchStreamLoad for jobId={}", jobId); - return new DorisBatchStreamLoad(jobId); + return new DorisBatchStreamLoad(jobId, targetDb); }); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java index de2eaf6ef19143..a2f9db236327f7 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java @@ -20,9 +20,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.io.Serializable; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; @@ -43,15 +43,14 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import org.apache.commons.codec.binary.Base64; +import lombok.Setter; import org.apache.commons.lang3.StringUtils; import org.apache.doris.cdcclient.common.Env; import org.apache.doris.cdcclient.exception.StreamLoadException; import org.apache.doris.cdcclient.utils.HttpUtil; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.http.HttpHeaders; import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpPut; +import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.util.EntityUtils; import org.slf4j.Logger; @@ -69,6 +68,7 @@ public class DorisBatchStreamLoad implements Serializable { private final int RETRY = 3; private final byte[] lineDelimiter = "\n".getBytes(); private static final String LOAD_URL_PATTERN = "http://%s/api/%s/%s/_stream_load"; + private static final String COMMIT_URL_PATTERN = "http://%s/api/%s/_stream_load_2pc"; private String hostPort; private Map bufferMap = new ConcurrentHashMap<>(); private ExecutorService loadExecutorService; @@ -82,8 +82,13 @@ public class DorisBatchStreamLoad implements Serializable { private final Lock lock = new ReentrantLock(); private final Condition block = lock.newCondition(); private final Map bufferMapLock = new ConcurrentHashMap<>(); + @Setter private String currentLabel; + private List successSubLabels = new ArrayList<>(); + private String targetDb; + private long jobId; + @Setter private String token; - public DorisBatchStreamLoad(long jobId) { + public DorisBatchStreamLoad(long jobId, String targetDb) { this.hostPort = Env.getCurrentEnv().getBackendHostPort(); this.flushQueue = new LinkedBlockingDeque<>(1); // maxBlockedBytes is two times of FLUSH_MAX_BYTE_SIZE @@ -100,6 +105,8 @@ public DorisBatchStreamLoad(long jobId) { new ThreadPoolExecutor.AbortPolicy()); this.started = new AtomicBoolean(true); this.loadExecutorService.execute(loadAsyncExecutor); + this.targetDb = targetDb; + this.jobId = jobId; } /** @@ -345,28 +352,25 @@ public void run() { /** execute stream load. */ public void load(String label, BatchRecordBuffer buffer) throws IOException { BatchBufferHttpEntity entity = new BatchBufferHttpEntity(buffer); - HttpPut put = - new HttpPut( - String.format( - LOAD_URL_PATTERN, - hostPort, - buffer.getDatabase(), - buffer.getTable())); - put.addHeader(HttpHeaders.EXPECT, "100-continue"); - put.addHeader("read_json_by_line", "true"); - put.addHeader("format", "json"); - put.addHeader( - HttpHeaders.AUTHORIZATION, - "Basic " - + new String( - Base64.encodeBase64("root:".getBytes(StandardCharsets.UTF_8)))); - put.setEntity(entity); + HttpPutBuilder putBuilder = new HttpPutBuilder(); + String loadUrl = String.format(LOAD_URL_PATTERN, hostPort, targetDb, buffer.getTable()); + putBuilder + .setUrl(loadUrl) + .addTokenAuth(token) + .setLabel(currentLabel) + .setSubLabel(label) + .formatJson() + .enable2PC() + .addCommonHeader() + .setEntity(entity) + .addHiddenColumns(true) + .setEntity(entity); Throwable resEx = new Throwable(); int retry = 0; while (retry <= RETRY) { try (CloseableHttpClient httpClient = HttpUtil.getHttpClient()) { - try (CloseableHttpResponse response = httpClient.execute(put)) { + try (CloseableHttpResponse response = httpClient.execute(putBuilder.build())) { int statusCode = response.getStatusLine().getStatusCode(); String reason = response.getStatusLine().toString(); if (statusCode == 200 && response.getEntity() != null) { @@ -375,6 +379,7 @@ public void load(String label, BatchRecordBuffer buffer) throws IOException { RespContent respContent = OBJECT_MAPPER.readValue(loadResult, RespContent.class); if (DORIS_SUCCESS_STATUS.contains(respContent.getStatus())) { + successSubLabels.add(label); long cacheByteBeforeFlush = currentCacheBytes.getAndAdd(-respContent.getLoadBytes()); LOG.info( @@ -453,13 +458,86 @@ public Thread newThread(Runnable r) { } } - @VisibleForTesting - public AtomicReference getException() { - return exception; + public void resetLabel() { + this.currentLabel = null; + this.successSubLabels.clear(); } - @VisibleForTesting - public boolean isLoadThreadAlive() { - return loadThreadAlive; + /** commit transaction. */ + public void commitTransaction(Map meta) { + try { + String url = String.format(COMMIT_URL_PATTERN, hostPort, targetDb); + + Map attachment = new HashMap<>(); + attachment.put("offset", OBJECT_MAPPER.writeValueAsString(meta)); + Map commitParams = new HashMap<>(); + commitParams.put("attachment", attachment); + commitParams.put("sub_labels", successSubLabels); + HttpPutBuilder builder = + new HttpPutBuilder() + .addCommonHeader() + .addTokenAuth(token) + .setLabel(currentLabel) + .setUrl(url) + .commit() + .setEntity( + new StringEntity( + OBJECT_MAPPER.writeValueAsString(commitParams))); + + LOG.info("commit transaction with label: {}", currentLabel); + Throwable resEx = null; + int retry = 0; + while (retry <= RETRY) { + try (CloseableHttpClient httpClient = HttpUtil.getHttpClient()) { + try (CloseableHttpResponse httpResponse = httpClient.execute(builder.build())) { + int statusCode = httpResponse.getStatusLine().getStatusCode(); + String reason = httpResponse.getStatusLine().toString(); + String responseBody = + httpResponse.getEntity() != null + ? EntityUtils.toString(httpResponse.getEntity()) + : ""; + LOG.info("commit result {}", responseBody); + if (statusCode == 200) { + LOG.info("commit transaction success, label: {}", currentLabel); + return; + } + LOG.error( + "commit transaction failed with {}, reason {}, to retry", + hostPort, + reason); + if (retry == RETRY) { + resEx = + new StreamLoadException( + "commit transaction failed with: " + reason); + } + } catch (Exception ex) { + resEx = ex; + LOG.error( + "commit transaction error with {}, to retry, cause by", + hostPort, + ex); + } + } + retry++; + if (retry <= RETRY) { + try { + Thread.sleep(retry * 1000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + } + + if (retry > RETRY) { + throw new StreamLoadException( + "commit transaction error: " + + (resEx != null ? resEx.getMessage() : "unknown error"), + resEx); + } + } catch (Exception ex) { + LOG.error("Failed to commit transaction, jobId={}", jobId, ex); + throw new StreamLoadException("Failed to commit transaction", ex); + } } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java new file mode 100644 index 00000000000000..39b5699b62d07d --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java @@ -0,0 +1,135 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import org.apache.doris.cdcclient.constants.LoadConstants; +import org.apache.flink.util.Preconditions; +import org.apache.http.HttpEntity; +import org.apache.http.HttpHeaders; +import org.apache.http.client.methods.HttpPut; +import org.apache.http.entity.StringEntity; + +/** Builder for HttpPut. */ +public class HttpPutBuilder { + String url; + Map header; + HttpEntity httpEntity; + + public HttpPutBuilder() { + header = new HashMap<>(); + } + + public HttpPutBuilder setUrl(String url) { + this.url = url; + return this; + } + + public HttpPutBuilder addCommonHeader() { + header.put(HttpHeaders.EXPECT, "100-continue"); + return this; + } + + public HttpPutBuilder addHiddenColumns(boolean add) { + if (add) { + header.put("hidden_columns", LoadConstants.DELETE_SIGN_KEY); + } + return this; + } + + public HttpPutBuilder enable2PC() { + header.put("two_phase_commit", "true"); + return this; + } + + public HttpPutBuilder addTokenAuth(String token) { + header.put(HttpHeaders.AUTHORIZATION, "Basic YWRtaW46"); + header.put("token", token); + return this; + } + + public HttpPutBuilder addTxnId(long txnID) { + header.put("txn_id", String.valueOf(txnID)); + return this; + } + + public HttpPutBuilder formatJson() { + header.put("read_json_by_line", "true"); + header.put("format", "json"); + return this; + } + + public HttpPutBuilder commit() { + header.put("txn_operation", "commit"); + return this; + } + + public HttpPutBuilder abort() { + header.put("txn_operation", "abort"); + return this; + } + + public HttpPutBuilder setEntity(HttpEntity httpEntity) { + this.httpEntity = httpEntity; + return this; + } + + public HttpPutBuilder setEmptyEntity() { + try { + this.httpEntity = new StringEntity(""); + } catch (Exception e) { + throw new IllegalArgumentException(e); + } + return this; + } + + public HttpPutBuilder addProperties(Properties properties) { + // TODO: check duplicate key. + properties.forEach((key, value) -> header.put(String.valueOf(key), String.valueOf(value))); + return this; + } + + public HttpPutBuilder setLabel(String label) { + if (label != null) { + header.put("label", label); + } + return this; + } + + public HttpPutBuilder setSubLabel(String subLabel) { + if (subLabel != null) { + header.put("sub_label", subLabel); + } + return this; + } + + public String getLabel() { + return header.get("label"); + } + + public HttpPut build() { + Preconditions.checkNotNull(url); + Preconditions.checkNotNull(httpEntity); + HttpPut put = new HttpPut(url); + header.forEach(put::setHeader); + put.setEntity(httpEntity); + return put; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java index e28d61ca90a778..e5e387df6f4e0a 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java @@ -17,6 +17,8 @@ package org.apache.doris.cdcclient.source.deserialize; +import static org.apache.doris.cdcclient.constants.LoadConstants.DELETE_SIGN_KEY; + import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import io.debezium.data.Envelope; @@ -52,7 +54,6 @@ public class DebeziumJsonDeserializer implements SourceRecordDeserializer> { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(DebeziumJsonDeserializer.class); - private static final String DELETE_SIGN_KEY = "__DORIS_DELETE_SIGN__"; private static ObjectMapper objectMapper = new ObjectMapper(); public DebeziumJsonDeserializer() {} @@ -110,7 +111,6 @@ private String extractAfterRow(Struct value, Schema valueSchema) record.put(field.name(), valueConverted); }); record.put(DELETE_SIGN_KEY, 0); - System.out.println(record); return objectMapper.writeValueAsString(record); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index 1428c579d2abb7..1d820877a42ab6 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -19,6 +19,7 @@ import java.util.List; import java.util.Map; +import org.apache.doris.cdcclient.model.JobConfig; import org.apache.doris.cdcclient.model.request.FetchRecordReq; import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; import org.apache.doris.cdcclient.model.request.JobBaseRecordReq; @@ -67,4 +68,6 @@ default SplitReadResult readSplitRecords(JobBaseRecordReq bas void close(Long jobId); void finishSplitRecords(); + + Map getEndOffset(JobConfig jobConfig); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index edbf2598d98fb6..c2790a7e483a39 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -20,7 +20,6 @@ import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.github.shyiko.mysql.binlog.BinaryLogClient; import io.debezium.connector.mysql.MySqlConnection; @@ -88,7 +87,6 @@ public class MySqlSourceReader implements SourceReader getSourceSplits(FetchTableSplitsReq ftsReq) { } } else { BinlogOffset startingOffset = remainingBinlogSplit.getStartingOffset(); - BinlogSplit binlogSplit = - new BinlogSplit(remainingBinlogSplit.splitId(), startingOffset.getOffset()); + BinlogSplit binlogSplit = new BinlogSplit(); + binlogSplit.setSplitId(remainingBinlogSplit.splitId()); + binlogSplit.setStartingOffset(startingOffset.getOffset()); splits.add(binlogSplit); } return splits; @@ -158,7 +157,7 @@ public RecordWithMeta read(FetchRecordReq fetchRecord) throws Exception { @Override public SplitReadResult readSplitRecords(JobBaseRecordReq baseReq) throws Exception { - Map offsetMeta = baseReq.getMeta(); + Map offsetMeta = baseReq.getMeta(); if (offsetMeta == null || offsetMeta.isEmpty()) { throw new RuntimeException("miss meta offset"); } @@ -258,10 +257,13 @@ private RecordWithMeta buildRecordResponse( Map offsetRes = highWatermark.getOffset(); offsetRes.put(SPLIT_ID, split.splitId()); recordResponse.setMeta(offsetRes); + return recordResponse; } if (CollectionUtils.isEmpty(recordResponse.getRecords())) { if (readBinlog) { - Map offsetRes = new HashMap<>(fetchRecord.getMeta()); + BinlogSplit binlogSplit = + objectMapper.convertValue(fetchRecord.getMeta(), BinlogSplit.class); + Map offsetRes = binlogSplit.getStartingOffset(); if (split != null) { offsetRes = split.asBinlogSplit().getStartingOffset().getOffset(); } @@ -269,7 +271,12 @@ private RecordWithMeta buildRecordResponse( offsetRes.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); recordResponse.setMeta(offsetRes); } else { - recordResponse.setMeta(fetchRecord.getMeta()); + SnapshotSplit snapshotSplit = + objectMapper.convertValue(fetchRecord.getMeta(), SnapshotSplit.class); + Map meta = new HashMap<>(); + meta.put(SPLIT_ID, snapshotSplit.getSplitId()); + // chunk no data + recordResponse.setMeta(meta); } } return recordResponse; @@ -297,22 +304,22 @@ private void refreshTableChanges(SourceRecord element, Long jobId) throws IOExce } private Tuple2 createMySqlSplit( - Map offset, JobConfig jobConfig) throws JsonProcessingException { + Map offsetMeta, JobConfig jobConfig) throws JsonProcessingException { Tuple2 splitRes = null; - String splitId = offset.get(SPLIT_ID); + String splitId = String.valueOf(offsetMeta.get(SPLIT_ID)); if (!BINLOG_SPLIT_ID.equals(splitId)) { - MySqlSnapshotSplit split = createSnapshotSplit(offset, jobConfig); + MySqlSnapshotSplit split = createSnapshotSplit(offsetMeta, jobConfig); splitRes = Tuple2.of(split, false); } else { - splitRes = createBinlogSplit(offset, jobConfig); + splitRes = createBinlogSplit(offsetMeta, jobConfig); } return splitRes; } - private MySqlSnapshotSplit createSnapshotSplit(Map offset, JobConfig jobConfig) + private MySqlSnapshotSplit createSnapshotSplit(Map offset, JobConfig jobConfig) throws JsonProcessingException { - String splitId = offset.get(SPLIT_ID); - SnapshotSplit snapshotSplit = SnapshotSplit.fromMap(offset); + // SnapshotSplit snapshotSplit = SnapshotSplit.fromMap(offset); + SnapshotSplit snapshotSplit = objectMapper.convertValue(offset, SnapshotSplit.class); TableId tableId = TableId.parse(snapshotSplit.getTableId()); Object[] splitStart = snapshotSplit.getSplitStart(); Object[] splitEnd = snapshotSplit.getSplitEnd(); @@ -327,28 +334,35 @@ private MySqlSnapshotSplit createSnapshotSplit(Map offset, JobCo RowType splitType = ChunkUtils.getChunkKeyColumnType(splitColumn); MySqlSnapshotSplit split = new MySqlSnapshotSplit( - tableId, splitId, splitType, splitStart, splitEnd, null, tableSchemas); + tableId, + snapshotSplit.getSplitId(), + splitType, + splitStart, + splitEnd, + null, + tableSchemas); return split; } private Tuple2 createBinlogSplit( - Map meta, JobConfig config) throws JsonProcessingException { + Map meta, JobConfig config) throws JsonProcessingException { MySqlSourceConfig sourceConfig = getSourceConfig(config); BinlogOffset offsetConfig = null; if (sourceConfig.getStartupOptions() != null) { offsetConfig = sourceConfig.getStartupOptions().binlogOffset; } - + BinlogSplit binlogSplit = objectMapper.convertValue(meta, BinlogSplit.class); List finishedSnapshotSplitInfos = new ArrayList<>(); BinlogOffset minOffsetFinishSplits = null; BinlogOffset maxOffsetFinishSplits = null; - if (meta.containsKey(FINISH_SPLITS)) { // List + if (CollectionUtils.isNotEmpty(binlogSplit.getFinishedSplits())) { + // if (meta.containsKey(FINISH_SPLITS)) { // List // Construct binlogsplit based on the finished split - String finishSplitsOffset = meta.remove(FINISH_SPLITS); - List splitWithHW = - objectMapper.readValue( - finishSplitsOffset, new TypeReference>() {}); - + // String finishSplitsOffset = meta.remove(FINISH_SPLITS); + // List splitWithHW = + // objectMapper.readValue( + // finishSplitsOffset, new TypeReference>() {}); + List splitWithHW = binlogSplit.getFinishedSplits(); List assignedSplitLists = splitWithHW.stream() .sorted(Comparator.comparing(AbstractSourceSplit::getSplitId)) @@ -375,7 +389,11 @@ private Tuple2 createBinlogSplit( } BinlogOffset startOffset; - BinlogOffset lastOffset = new BinlogOffset(meta); + BinlogOffset lastOffset = + new BinlogOffset( + binlogSplit.getStartingOffset() == null + ? new HashMap<>() + : binlogSplit.getStartingOffset()); if (minOffsetFinishSplits != null && lastOffset.getOffsetKind() == null) { startOffset = minOffsetFinishSplits; } else if (lastOffset.getOffsetKind() != null && lastOffset.getFilename() != null) { @@ -407,9 +425,9 @@ private Tuple2 createBinlogSplit( new HashMap<>(), 0); // filterTableSchema - MySqlBinlogSplit binlogSplit = + MySqlBinlogSplit binlogSplitFinal = MySqlBinlogSplit.fillTableSchemas(split.asBinlogSplit(), getTableSchemas(config)); - return Tuple2.of(binlogSplit, pureBinlogPhase); + return Tuple2.of(binlogSplitFinal, pureBinlogPhase); } private List startSplitChunks( @@ -584,6 +602,17 @@ public void finishSplitRecords() { jobRuntimeContext.setCurrentSplitRecords(null); } + @Override + public Map getEndOffset(JobConfig jobConfig) { + MySqlSourceConfig sourceConfig = getSourceConfig(jobConfig); + try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) { + BinlogOffset binlogOffset = DebeziumUtils.currentBinlogOffset(jdbc); + return binlogOffset.getOffset(); + } catch (SQLException ex) { + throw new RuntimeException(ex); + } + } + private Map getTableSchemas(JobConfig config) { Map schemas = jobRuntimeContext.getTableSchemas(); if (schemas == null) { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java index 5ed92d4c56fe97..6141e05bbc301e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java @@ -17,6 +17,7 @@ package org.apache.doris.cdcclient.source.split; +import java.io.Serializable; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; @@ -26,7 +27,11 @@ @Setter @AllArgsConstructor @NoArgsConstructor -public abstract class AbstractSourceSplit implements SourceSplit { +public abstract class AbstractSourceSplit implements SourceSplit, Serializable { private static final long serialVersionUID = 1L; protected String splitId; + + public boolean snapshotSplit() { + return this instanceof SnapshotSplit; + } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java index 8d31ec46d53b6e..2b261bc4939b45 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java @@ -17,20 +17,21 @@ package org.apache.doris.cdcclient.source.split; +import java.util.List; import java.util.Map; import lombok.EqualsAndHashCode; import lombok.Getter; +import lombok.NoArgsConstructor; import lombok.Setter; @Getter @Setter @EqualsAndHashCode(callSuper = true) +@NoArgsConstructor public class BinlogSplit extends AbstractSourceSplit { private static final long serialVersionUID = 1L; - private Map offset; - - public BinlogSplit(String splitId, Map offset) { - super(splitId); - this.offset = offset; - } + private Map startingOffset; + private Map endingOffset; + // binlog split meta, first binlog split requires + private List finishedSplits; } From 3c2d358af69bf647e27ab239f9e411a7c158376c Mon Sep 17 00:00:00 2001 From: JNSimba <676366545@qq.com> Date: Fri, 5 Dec 2025 10:26:04 +0800 Subject: [PATCH 09/27] fix be forward request --- be/src/common/config.cpp | 2 - be/src/common/config.h | 3 - be/src/runtime/cdc_client_manager.cpp | 168 ++------------------------ be/src/runtime/cdc_client_manager.h | 18 +-- be/src/runtime/exec_env_init.cpp | 2 +- be/src/service/internal_service.cpp | 14 --- be/src/service/internal_service.h | 7 +- gensrc/proto/internal_service.proto | 2 - 8 files changed, 11 insertions(+), 205 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 21d8f17ce3b8d8..b955778c108514 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -755,8 +755,6 @@ DEFINE_mInt32(max_consumer_num_per_group, "3"); // this should be larger than FE config 'max_routine_load_task_num_per_be' (default 5) DEFINE_Int32(max_routine_load_thread_pool_size, "1024"); -DEFINE_mInt32(max_cdc_client_thread_pool_size, "128"); - // the timeout of condition variable wait in blocking_get and blocking_put DEFINE_mInt32(blocking_queue_cv_wait_timeout_ms, "1000"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 490264e9b7ef74..9101f3e5a0a7ac 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -781,9 +781,6 @@ DECLARE_mInt32(max_consumer_num_per_group); // this should be larger than FE config 'max_routine_load_task_num_per_be' (default 5) DECLARE_Int32(max_routine_load_thread_pool_size); -// Thread pool size for CDC client manager -DECLARE_mInt32(max_cdc_client_thread_pool_size); - // max external scan cache batch count, means cache max_memory_cache_batch_count * batch_size row // default is 20, batch_size's default value is 1024 means 20 * 1024 rows will be cached DECLARE_mInt32(max_memory_sink_batch_count); diff --git a/be/src/runtime/cdc_client_manager.cpp b/be/src/runtime/cdc_client_manager.cpp index 3a0d176c140540..6bdf895ac7c6ab 100644 --- a/be/src/runtime/cdc_client_manager.cpp +++ b/be/src/runtime/cdc_client_manager.cpp @@ -19,14 +19,8 @@ #include #include -#include -#include -#include #include #include -#include -#include -#include #include #include #include @@ -34,21 +28,12 @@ #include #endif -#include #include #include "common/config.h" #include "common/logging.h" #include "common/status.h" #include "http/http_client.h" -#include "runtime/cluster_info.h" -#include "runtime/exec_env.h" -#include "service/backend_options.h" -#include "util/thrift_util.h" -#include "util/threadpool.h" -#include -#include -#include "runtime/client_cache.h" namespace doris { @@ -62,7 +47,7 @@ void handle_sigchld(int sig_no) { // Check CDC client health Status check_cdc_client_health(int retry_times, int sleep_time, std::string& health_response) { - const std::string cdc_health_url = "http://" + BackendOptions::get_localhost() + ":" + + const std::string cdc_health_url = "http://127.0.0.1:" + std::to_string(doris::config::cdc_client_port) + "/actuator/health"; @@ -91,7 +76,7 @@ Status check_cdc_client_health(int retry_times, int sleep_time, std::string& hea } // Start CDC client process -Status start_cdc_client(const std::string& params, PRequestCdcClientResult* result) { +Status start_cdc_client(PRequestCdcClientResult* result) { Status st = Status::OK(); // Check DORIS_HOME environment variable @@ -116,9 +101,7 @@ Status start_cdc_client(const std::string& params, PRequestCdcClientResult* resu const std::string cdc_jar_path = std::string(doris_home) + "/lib/cdc_client/cdc-client.jar"; const std::string cdc_jar_port = "--server.port=" + std::to_string(doris::config::cdc_client_port); - const std::string backend_host_port = - BackendOptions::get_localhost() + ":" + std::to_string(config::webserver_port); - const std::string cdc_jar_params = params; + const std::string backend_http_port = std::to_string(config::webserver_port); const std::string java_opts = "-Xmx2048m -Dlog.path=" + std::string(log_dir); // check cdc jar exists @@ -177,9 +160,9 @@ Status start_cdc_client(const std::string& params, PRequestCdcClientResult* resu << std::endl; std::cout << "Cdc client child process ready to start." << std::endl; std::string java_bin = path + "/bin/java"; + // java -jar -Dlog.path=xx cdc-client.jar --server.port=9096 8040 execlp(java_bin.c_str(), "java", java_opts.c_str(), "-jar", cdc_jar_path.c_str(), - cdc_jar_port.c_str(), backend_host_port.c_str(), cdc_jar_params.c_str(), - (char*)NULL); + cdc_jar_port.c_str(), backend_http_port.c_str(),(char*)NULL); std::cerr << "Cdc client child process error." << std::endl; exit(1); } else { @@ -203,12 +186,7 @@ Status start_cdc_client(const std::string& params, PRequestCdcClientResult* resu } // anonymous namespace -CdcClientManager::CdcClientManager(ExecEnv* exec_env) : _exec_env(exec_env) { - static_cast(ThreadPoolBuilder("CdcClientThreadPool") - .set_min_threads(1) - .set_max_threads(config::max_cdc_client_thread_pool_size) - .build(&_thread_pool)); -} +CdcClientManager::CdcClientManager() = default; CdcClientManager::~CdcClientManager() { stop(); @@ -216,9 +194,6 @@ CdcClientManager::~CdcClientManager() { } void CdcClientManager::stop() { - if (_thread_pool) { - _thread_pool->shutdown(); - } LOG(INFO) << "CdcClientManager is stopped"; } @@ -229,14 +204,13 @@ void CdcClientManager::request_cdc_client_impl(const PRequestCdcClientRequest* r brpc::ClosureGuard closure_guard(done); // Start CDC client if not started - Status start_st = start_cdc_client(request->params(), nullptr); + Status start_st = start_cdc_client(result); if (!start_st.ok()) { LOG(ERROR) << "Failed to start CDC client, status=" << start_st.to_string(); start_st.to_protobuf(result->mutable_status()); return; } - // Send HTTP request synchronously (this is called from heavy_work_pool, so it's already async) std::string cdc_response; Status st = send_request_to_cdc_client(request->api(), request->params(), &cdc_response); result->set_response(cdc_response); @@ -247,8 +221,7 @@ void CdcClientManager::request_cdc_client_impl(const PRequestCdcClientRequest* r Status CdcClientManager::send_request_to_cdc_client(const std::string& api, const std::string& params_body, std::string* response) { - std::string remote_url_prefix = fmt::format("http://{}:{}{}", - BackendOptions::get_localhost(), + std::string remote_url_prefix = fmt::format("http://127.0.0.1:{}{}", doris::config::cdc_client_port, api); @@ -267,130 +240,5 @@ Status CdcClientManager::send_request_to_cdc_client(const std::string& api, return HttpClient::execute_with_retry(3, 1, cdc_request); } -Status CdcClientManager::extract_meta_from_response(const std::string& cdc_response, - std::string* meta_json) { - rapidjson::Document doc; - if (doc.Parse(cdc_response.c_str()).HasParseError()) { - return Status::InternalError("Failed to parse CDC response JSON"); - } - - // Check if there is a data field - if (!doc.HasMember("data") || !doc["data"].IsObject()) { - return Status::InternalError("CDC response missing 'data' field or not an object"); - } - - // Check if there is a meta field - const rapidjson::Value& data = doc["data"]; - if (!data.HasMember("meta") || !data["meta"].IsObject()) { - return Status::InternalError("CDC response missing 'meta' field or not an object"); - } - - // Extract meta object and serialize as JSON string - const rapidjson::Value& meta = data["meta"]; - rapidjson::StringBuffer buffer; - rapidjson::Writer writer(buffer); - meta.Accept(writer); - *meta_json = buffer.GetString(); - return Status::OK(); -} - -Status CdcClientManager::commit_transaction(int64_t txn_id, - const std::string& meta_json) { - TNetworkAddress master_addr = _exec_env->cluster_info()->master_fe_addr; - /** - TLoadTxnCommitRequest commit_request; - commit_request.__set_txnId(txn_id); - - - TNetworkAddress master_addr = _exec_env->cluster_info()->master_fe_addr; - TLoadTxnCommitResult commit_result; - - Status rpc_st = ThriftRpcHelper::rpc( - master_addr.hostname, master_addr.port, - [&commit_request, &commit_result](FrontendServiceConnection& client) { - client->loadTxnCommit(commit_result, commit_request); - }, - config::txn_commit_rpc_timeout_ms); - - if (!rpc_st.ok()) { - return Status::InternalError( - fmt::format("Failed to call FE loadTxnCommit, rpc_status={}, txn_id={}", - rpc_st.to_string(), txn_id)); - } - - Status result_status = Status::create(commit_result.status); - if (!result_status.ok()) { - return Status::InternalError( - fmt::format("FE loadTxnCommit returned error, status={}, txn_id={}", - result_status.to_string(), txn_id)); - } - */ - return Status::OK(); -} - -void CdcClientManager::execute_cdc_scan_commit_impl(const PRequestCdcClientRequest* request, - PRequestCdcClientResult* result, - google::protobuf::Closure* done) { - VLOG_RPC << "request to cdc client, api " << request->api(); - brpc::ClosureGuard closure_guard(done); - - // Start CDC client if not started - Status start_st = start_cdc_client(request->params(), nullptr); - if (!start_st.ok()) { - LOG(ERROR) << "Failed to start CDC client, status=" << start_st.to_string(); - start_st.to_protobuf(result->mutable_status()); - return; - } - - // Extract parameters from request - std::string api = request->api(); - int64_t txn_id = request->txn_id(); - std::string params_body = request->params(); - - // Submit async task to handle CDC scan and commit using internal thread_pool - Status submit_st = _thread_pool->submit_func([this, api, params_body, txn_id]() { - // Request cdc client to read and load data - std::string cdc_response; - Status st = send_request_to_cdc_client(api, params_body, &cdc_response); - if (!st.ok()) { - LOG(ERROR) << "CDC client HTTP request failed, status=" << st.to_string() - << ", api=" << api << ", txn_id=" << txn_id; - return; - } - - LOG(INFO) << "CDC client HTTP request success, response=" << cdc_response - << ", txn_id=" << txn_id; - - // Parse JSON, extract data.meta part - std::string meta_json; - Status parse_st = extract_meta_from_response(cdc_response, &meta_json); - if (!parse_st.ok()) { - LOG(ERROR) << "Failed to extract meta from CDC response, txn_id=" << txn_id - << ", status=" << parse_st.to_string(); - return; - } - - // Commit txn - Status commit_st = commit_transaction(txn_id, meta_json); - if (!commit_st.ok()) { - LOG(ERROR) << "Failed to commit CDC transaction, txn_id=" << txn_id - << ", status=" << commit_st.to_string(); - return; - } - - LOG(INFO) << "Successfully committed CDC transaction to FE, txn_id=" << txn_id; - }); - - if (!submit_st.ok()) { - LOG(ERROR) << "Failed to submit CDC client async task to thread pool, " - << "status=" << submit_st.to_string() << ", txn_id=" << txn_id; - submit_st.to_protobuf(result->mutable_status()); - return; - } - - // Return success to FE immediately after task is successfully submitted - Status::OK().to_protobuf(result->mutable_status()); -} - } // namespace doris diff --git a/be/src/runtime/cdc_client_manager.h b/be/src/runtime/cdc_client_manager.h index bdeb641e1b8382..9c97a6a5e6b6e4 100644 --- a/be/src/runtime/cdc_client_manager.h +++ b/be/src/runtime/cdc_client_manager.h @@ -22,7 +22,6 @@ #include #include "common/status.h" -#include "util/threadpool.h" namespace google::protobuf { class Closure; @@ -31,11 +30,9 @@ class RpcController; namespace doris { -class ExecEnv; - class CdcClientManager { public: - explicit CdcClientManager(ExecEnv* exec_env); + CdcClientManager(); ~CdcClientManager(); void stop(); @@ -45,23 +42,10 @@ class CdcClientManager { PRequestCdcClientResult* result, google::protobuf::Closure* done); - // Execute CDC scan and commit transaction - void execute_cdc_scan_commit_impl(const PRequestCdcClientRequest* request, - PRequestCdcClientResult* result, - google::protobuf::Closure* done); - private: Status send_request_to_cdc_client(const std::string& api, const std::string& params_body, std::string* response); - - Status extract_meta_from_response(const std::string& cdc_response, - std::string* meta_json); - - Status commit_transaction(const int64_t txn_id, const std::string& meta_json); - - ExecEnv* _exec_env = nullptr; - std::unique_ptr _thread_pool; }; } // namespace doris diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 26926354d24e9f..bffe51de7d140e 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -336,7 +336,7 @@ Status ExecEnv::_init(const std::vector& store_paths, RETURN_IF_ERROR(_routine_load_task_executor->init(MemInfo::mem_limit())); _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); _group_commit_mgr = new GroupCommitMgr(this); - _cdc_client_mgr = new CdcClientManager(this); + _cdc_client_mgr = new CdcClientManager(); _memtable_memory_limiter = std::make_unique(); _load_stream_map_pool = std::make_unique(); _delta_writer_v2_pool = std::make_unique(); diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index e8acddcc4b22b3..1ddfaeb1759693 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -2411,19 +2411,5 @@ void PInternalService::request_cdc_client(google::protobuf::RpcController* contr } } -void PInternalService::execute_cdc_scan_commit(google::protobuf::RpcController* controller, - const PRequestCdcClientRequest* request, - PRequestCdcClientResult* result, - google::protobuf::Closure* done) { - bool ret = _heavy_work_pool.try_offer([this, request, result, done]() { - _exec_env->cdc_client_mgr()->execute_cdc_scan_commit_impl(request, result, done); - }); - - if (!ret) { - offer_failed(result, done, _heavy_work_pool); - return; - } -} - #include "common/compile_check_avoid_end.h" } // namespace doris diff --git a/be/src/service/internal_service.h b/be/src/service/internal_service.h index 1a9ad77308ac98..792c953d975b20 100644 --- a/be/src/service/internal_service.h +++ b/be/src/service/internal_service.h @@ -235,12 +235,7 @@ class PInternalService : public PBackendService { void request_cdc_client(google::protobuf::RpcController* controller, const PRequestCdcClientRequest* request, PRequestCdcClientResult* result, - google::protobuf::Closure* done) override; - - void execute_cdc_scan_commit(google::protobuf::RpcController* controller, - const PRequestCdcClientRequest* request, - PRequestCdcClientResult* result, - google::protobuf::Closure* done) override; + google::protobuf::Closure* done) override; private: void _exec_plan_fragment_in_pthread(google::protobuf::RpcController* controller, diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index 1728f88dc1ea3d..f15993b1dbf5d9 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -1120,7 +1120,6 @@ message PFetchPeerDataResponse { message PRequestCdcClientRequest { optional string api = 1; optional string params = 2; - optional int64 txn_id = 3; } message PRequestCdcClientResult { @@ -1186,6 +1185,5 @@ service PBackendService { rpc get_tablet_rowsets(PGetTabletRowsetsRequest) returns (PGetTabletRowsetsResponse); rpc fetch_peer_data(PFetchPeerDataRequest) returns (PFetchPeerDataResponse); rpc request_cdc_client(PRequestCdcClientRequest) returns (PRequestCdcClientResult); - rpc execute_cdc_scan_commit(PRequestCdcClientRequest) returns (PRequestCdcClientResult); }; From 124d2ae8f8f682bf84c95c52ab472a88497b3c1c Mon Sep 17 00:00:00 2001 From: wudi Date: Fri, 5 Dec 2025 23:01:14 +0800 Subject: [PATCH 10/27] add multi table sync and checkstyle --- .../apache/doris/httpv2/rest/LoadAction.java | 2 +- .../doris/httpv2/rest/StreamingJobAction.java | 83 ++++ .../doris/job/common/DataSourceType.java | 17 + .../streaming/AbstractStreamingTask.java | 24 +- .../insert/streaming/StreamingInsertJob.java | 100 +++-- .../insert/streaming/StreamingInsertTask.java | 11 - .../streaming/StreamingMultiTblTask.java | 140 +++---- .../job/offset/SourceOffsetProvider.java | 23 ++ .../doris/job/offset/jdbc/JdbcOffset.java | 23 +- .../offset/jdbc/JdbcSourceOffsetProvider.java | 377 ++++++++++++------ .../jdbc/split/AbstractSourceSplit.java | 2 +- .../job/offset/jdbc/split/BinlogSplit.java | 11 +- .../job/offset/jdbc/split/SnapshotSplit.java | 51 +-- .../job/scheduler/StreamingTaskScheduler.java | 1 - .../doris/job/util/StreamingJobUtils.java | 98 ++++- .../nereids/parser/LogicalPlanBuilder.java | 2 +- .../plans/commands/info/CreateJobInfo.java | 24 +- .../doris/rpc/BackendServiceClient.java | 5 - .../apache/doris/rpc/BackendServiceProxy.java | 13 +- .../tablefunction/MetadataGenerator.java | 1 - .../controller/ClientController.java | 18 +- .../model/request/CompareOffsetReq.java | 30 ++ .../model/request/WriteRecordReq.java | 4 +- .../service/PipelineCoordinator.java | 36 +- .../cdcclient/sink/DorisBatchStreamLoad.java | 79 ++-- .../doris/cdcclient/sink/HttpPutBuilder.java | 7 - .../cdcclient/source/reader/SourceReader.java | 2 + .../reader/mysql/MySqlSourceReader.java | 7 + 28 files changed, 778 insertions(+), 413 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StreamingJobAction.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java index 4fcd432051f7ee..890a41c5710ee1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java @@ -595,7 +595,7 @@ private Pair splitHostAndPort(String hostPort) throws AnalysisE // AuditlogPlugin should be re-disigned carefully, and blow method focuses on // temporarily addressing the users' needs for audit logs. // So this function is not widely tested under general scenario - private boolean checkClusterToken(String token) { + protected boolean checkClusterToken(String token) { try { return Env.getCurrentEnv().getTokenManager().checkAuthToken(token); } catch (UserException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StreamingJobAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StreamingJobAction.java new file mode 100644 index 00000000000000..ff96bf9365c753 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StreamingJobAction.java @@ -0,0 +1,83 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.httpv2.rest; + +import org.apache.doris.catalog.Env; +import org.apache.doris.httpv2.entity.ResponseEntityBuilder; +import org.apache.doris.httpv2.exception.UnauthorizedException; +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; + +import com.google.common.base.Strings; +import jakarta.servlet.http.HttpServletRequest; +import lombok.Getter; +import lombok.Setter; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; + +public class StreamingJobAction extends LoadAction { + private static final Logger LOG = LogManager.getLogger(StreamingJobAction.class); + + @RequestMapping(path = "/api/streaming/commit_offset", method = RequestMethod.PUT) + public Object streamLoad(HttpServletRequest request, @RequestBody CommitOffsetRequest offsetRequest) { + String authToken = request.getHeader("token"); + // if auth token is not null, check it first + if (!Strings.isNullOrEmpty(authToken)) { + if (!checkClusterToken(authToken)) { + throw new UnauthorizedException("Invalid token: " + authToken); + } + return updateOffset(offsetRequest); + } else { + // only use for token + throw new UnauthorizedException("Miss token"); + } + } + + private Object updateOffset(CommitOffsetRequest offsetRequest) { + AbstractJob job = Env.getCurrentEnv().getJobManager().getJob(offsetRequest.getJobId()); + if (job == null) { + String errMsg = "Job " + offsetRequest.getJobId() + " not found"; + return ResponseEntityBuilder.okWithCommonError(errMsg); + } + if (!(job instanceof StreamingInsertJob)) { + return ResponseEntityBuilder + .okWithCommonError("Job " + offsetRequest.getJobId() + " is not a streaming job"); + } + + StreamingInsertJob streamingJob = (StreamingInsertJob) job; + try { + streamingJob.commitOffset(offsetRequest.getTaskId(), offsetRequest.getOffset()); + return ResponseEntityBuilder.ok("Offset committed successfully"); + } catch (Exception e) { + LOG.warn("Failed to commit offset for job {}, offset {}: {}", offsetRequest.getJobId(), + offsetRequest.getOffset(), e.getMessage()); + return ResponseEntityBuilder.okWithCommonError(e.getMessage()); + } + } + + @Getter + @Setter + public static class CommitOffsetRequest { + public long jobId; + public long taskId; + public String offset; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/DataSourceType.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/DataSourceType.java index cf6368aff6a256..4ba5670fbd3410 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/common/DataSourceType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/DataSourceType.java @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.job.common; public enum DataSourceType { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java index 393f2e3df6e835..6554c6d6b23103 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java @@ -1,9 +1,25 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.job.extensions.insert.streaming; import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.Env; import org.apache.doris.common.FeConstants; -import org.apache.doris.common.Status; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.job.base.Job; import org.apache.doris.job.common.TaskStatus; @@ -12,7 +28,6 @@ import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; -import org.apache.doris.thrift.TStatusCode; import lombok.Getter; import lombok.Setter; @@ -52,8 +67,11 @@ public AbstractStreamingTask(long jobId, long taskId) { } public abstract void before() throws Exception; - public abstract void run() throws JobException ; + + public abstract void run() throws JobException; + public abstract boolean onSuccess() throws JobException; + public abstract void closeOrReleaseResources(); public void execute() throws JobException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index a87ff6466c61dd..3e016b9bc14195 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -40,6 +40,7 @@ import org.apache.doris.job.common.IntervalUnit; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.JobType; +import org.apache.doris.job.common.LoadConstants; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; @@ -52,7 +53,6 @@ import org.apache.doris.job.util.StreamingJobUtils; import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.loadv2.LoadStatistic; -import org.apache.doris.load.routineload.RoutineLoadTaskInfo; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundTVFRelation; @@ -79,16 +79,7 @@ import lombok.Getter; import lombok.Setter; import lombok.extern.log4j.Log4j2; -import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.lang3.StringUtils; -import static org.apache.doris.job.common.LoadConstants.DATABASE; -import static org.apache.doris.job.common.LoadConstants.DRIVER_CLASS; -import static org.apache.doris.job.common.LoadConstants.DRIVER_URL; -import static org.apache.doris.job.common.LoadConstants.EXCLUDE_TABLES; -import static org.apache.doris.job.common.LoadConstants.INCLUDE_TABLES; -import static org.apache.doris.job.common.LoadConstants.JDBC_URL; -import static org.apache.doris.job.common.LoadConstants.PASSWORD; -import static org.apache.doris.job.common.LoadConstants.USER; + import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; @@ -128,6 +119,10 @@ public class StreamingInsertJob extends AbstractJob createTbls = createTableIfNotExists(); this.offsetProvider = new JdbcSourceOffsetProvider(); - JdbcSourceOffsetProvider rdsOffsetProvider = (JdbcSourceOffsetProvider)this.offsetProvider; + JdbcSourceOffsetProvider rdsOffsetProvider = (JdbcSourceOffsetProvider) this.offsetProvider; rdsOffsetProvider.setJobId(getJobId()); rdsOffsetProvider.setSourceType(dataSourceType); rdsOffsetProvider.setSourceProperties(sourceProperties); rdsOffsetProvider.splitChunks(createTbls); - }catch (Exception ex){ + } catch (Exception ex) { log.warn("init streaming job for {} failed", dataSourceType, ex); throw new RuntimeException(ex.getMessage()); } } private void checkRequiredSourceProperties() { - Preconditions.checkArgument(sourceProperties.get(JDBC_URL) != null, "jdbc_url is required property"); - Preconditions.checkArgument(sourceProperties.get(DRIVER_URL) != null, "driver_url is required property"); - Preconditions.checkArgument(sourceProperties.get(DRIVER_CLASS) != null, "driver_class is required property"); - Preconditions.checkArgument(sourceProperties.get(USER) != null, "user is required property"); - Preconditions.checkArgument(sourceProperties.get(PASSWORD) != null, "password is required property"); - Preconditions.checkArgument(sourceProperties.get(DATABASE) != null, "database is required property"); - Preconditions.checkArgument(sourceProperties.get(INCLUDE_TABLES) != null || sourceProperties.get(EXCLUDE_TABLES) != null, + Preconditions.checkArgument(sourceProperties.get(LoadConstants.JDBC_URL) != null, + "jdbc_url is required property"); + Preconditions.checkArgument(sourceProperties.get(LoadConstants.DRIVER_URL) != null, + "driver_url is required property"); + Preconditions.checkArgument(sourceProperties.get(LoadConstants.DRIVER_CLASS) != null, + "driver_class is required property"); + Preconditions.checkArgument(sourceProperties.get(LoadConstants.USER) != null, + "user is required property"); + Preconditions.checkArgument(sourceProperties.get(LoadConstants.PASSWORD) != null, + "password is required property"); + Preconditions.checkArgument(sourceProperties.get(LoadConstants.DATABASE) != null, + "database is required property"); + Preconditions.checkArgument(sourceProperties.get(LoadConstants.INCLUDE_TABLES) != null + || sourceProperties.get(LoadConstants.EXCLUDE_TABLES) != null, "Either include_tables or exclude_tables must be specified"); } @@ -420,7 +422,7 @@ public List createTasks(TaskType taskType, Map sourceProperties; private Map targetProperties; private String targetDb; + private StreamingJobProperties jobProperties; public StreamingMultiTblTask(Long jobId, - long taskId, - DataSourceType dataSourceType, - SourceOffsetProvider offsetProvider, - Map sourceProperties, - String targetDb, - Map targetProperties) { + long taskId, + DataSourceType dataSourceType, + SourceOffsetProvider offsetProvider, + Map sourceProperties, + String targetDb, + Map targetProperties, + StreamingJobProperties jobProperties) { super(jobId, taskId); this.dataSourceType = dataSourceType; this.offsetProvider = offsetProvider; this.sourceProperties = sourceProperties; this.targetProperties = targetProperties; + this.jobProperties = jobProperties; } @Override @@ -77,74 +95,39 @@ public void run() throws JobException { } private void sendWriteRequest() throws JobException { + Backend backend = StreamingJobUtils.selectBackend(jobId); Map params = buildRequestParams(); - - String url = "http://127.0.0.1:9096/api/writeRecords"; - // Prepare request body - String requestBody = null; + InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() + .setApi("/api/writeRecords") + .setParams(GsonUtils.GSON.toJson(params)).build(); + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); + InternalService.PRequestCdcClientResult result = null; try { - requestBody = objectMapper.writeValueAsString(params); - }catch (IOException e) { - throw new JobException("Failed to serialize request body: " + e.getMessage(), e); - } - // Create HTTP POST request - HttpPost httpPost = new HttpPost(url); - httpPost.addHeader("token", getToken()); - StringEntity stringEntity = new StringEntity(requestBody, "UTF-8"); - stringEntity.setContentType("application/json"); - httpPost.setEntity(stringEntity); - // Set request headers - httpPost.setHeader(HttpHeaders.CONTENT_TYPE, "application/json"); - - // Execute request - try (CloseableHttpClient client = HttpClientBuilder.create().build()) { - String response = client.execute(httpPost, httpResponse -> { - int statusCode = httpResponse.getStatusLine().getStatusCode(); - String responseBody = EntityUtils.toString(httpResponse.getEntity(), "UTF-8"); - if (statusCode != 200) { - throw new RuntimeException("Failed to get split from CDC client, HTTP status code: " + statusCode); - } - return responseBody; - }); - + Future future = + BackendServiceProxy.getInstance().requestCdcClient(address, request); + result = future.get(); + TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + log.error("Failed to get split from backend, {}", result.getStatus().getErrorMsgs(0)); + throw new JobException( + "Failed to get split from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + + result.getResponse()); + } + String response = result.getResponse(); ResponseBody responseObj = objectMapper.readValue( response, - new TypeReference>() {} + new TypeReference>() { + } ); if (responseObj.getCode() == RestApiStatusCode.OK.code) { log.info("Send write records request successfully, response: {}", responseObj.getData()); return; } throw new JobException("Failed to send write records request , error message: " + responseObj); - } catch (Exception ex) { - log.error("Send write request: ", ex); - throw new JobException("Failed to send write request: " + ex.getMessage(), ex); + } catch (ExecutionException | InterruptedException | IOException ex) { + log.error("Send write request failed: ", ex); + throw new JobException(ex); } - - /** - Backend backend = selectBackend(jobId); - Map params = buildSplitParams(table); - InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() - .setApi("/api/fetchSplits") - .setParams(GsonUtils.GSON.toJson(params)).build(); - TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); - InternalService.PRequestCdcClientResult result = null; - try { - Future future = - BackendServiceProxy.getInstance().requestCdcClient(address, request); - result = future.get(); - TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); - if (code != TStatusCode.OK) { - log.error("Failed to get split from backend, {}", result.getStatus().getErrorMsgs(0)); - throw new JobException("Failed to get split from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + result.getResponse()); - } - } catch (ExecutionException | InterruptedException ex) { - log.error("Get splits error: ", ex); - throw new JobException(ex); - } - log.info("========fetch cdc split {}", result.getResponse()); - return ""; - **/ } private String getToken() throws JobException { @@ -159,7 +142,7 @@ private String getToken() throws JobException { return token; } - private Map buildRequestParams() { + private Map buildRequestParams() throws JobException { JdbcOffset offset = (JdbcOffset) runningOffset; Map params = new HashMap<>(); params.put("jobId", getJobId()); @@ -168,6 +151,11 @@ private Map buildRequestParams() { params.put("meta", offset.getSplit()); params.put("config", sourceProperties); params.put("targetDb", targetDb); + params.put("token", getToken()); + params.put("taskId", getTaskId()); + params.put("frontendAddress", + Env.getCurrentEnv().getMasterHost() + ":" + Env.getCurrentEnv().getMasterHttpPort()); + params.put("maxInterval", jobProperties.getMaxIntervalSecond()); return params; } @@ -197,7 +185,7 @@ public void closeOrReleaseResources() { } public boolean isTimeout() { - //todo: need to config + // todo: need to config return (System.currentTimeMillis() - createTimeMs) > 300 * 1000; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index c1460017d4c288..ccd0e422f82199 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -17,6 +17,7 @@ package org.apache.doris.job.offset; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; @@ -29,30 +30,35 @@ public interface SourceOffsetProvider { /** * Get source type, e.g. s3, kafka + * * @return */ String getSourceType(); /** * Get next offset to consume + * * @return */ Offset getNextOffset(StreamingJobProperties jobProps, Map properties); /** * Get current offset to show + * * @return */ String getShowCurrentOffset(); /** * Get remote datasource max offset to show + * * @return */ String getShowMaxOffset(); /** * Rewrite the TVF parameters in the SQL based on the current offset. + * * @param nextOffset * @return rewritten InsertIntoTableCommand */ @@ -60,6 +66,7 @@ public interface SourceOffsetProvider { /** * Update the offset of the source. + * * @param offset */ void updateOffset(Offset offset); @@ -71,21 +78,37 @@ public interface SourceOffsetProvider { /** * Whether there is more data to consume + * * @return */ boolean hasMoreDataToConsume(); /** * Deserialize string offset to Offset + * * @return */ Offset deserializeOffset(String offset); /** * Deserialize offset property to Offset + * * @return */ Offset deserializeOffsetProperty(String offset); + /** + * Replaying OffsetProvider is currently only required by JDBC. + * + * @return + */ + default void replayIfNeed(StreamingInsertJob job) { + } + + default String getPersistInfo() { + return null; + } + + ; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java index 160aa9e1d46915..da9c84bc909aa7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -1,13 +1,34 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.job.offset.jdbc; import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.jdbc.split.AbstractSourceSplit; + +import lombok.AllArgsConstructor; import lombok.Getter; +import lombok.NoArgsConstructor; import lombok.Setter; -import java.util.Map; @Getter @Setter +@AllArgsConstructor +@NoArgsConstructor public class JdbcOffset implements Offset { private AbstractSourceSplit split; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index 7717039ee173a1..f9ceb879c49436 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -1,37 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.job.offset.jdbc; -import org.apache.doris.catalog.Env; import org.apache.doris.httpv2.entity.ResponseBody; import org.apache.doris.job.common.DataSourceType; +import org.apache.doris.job.common.LoadConstants; import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.job.offset.jdbc.split.AbstractSourceSplit; import org.apache.doris.job.offset.jdbc.split.BinlogSplit; import org.apache.doris.job.offset.jdbc.split.SnapshotSplit; +import org.apache.doris.job.util.StreamingJobUtils; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.InternalService.PRequestCdcClientResult; +import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.system.Backend; -import org.apache.doris.system.BeSelectionPolicy; -import org.apache.doris.system.SystemInfoService; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TStatusCode; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import com.google.gson.annotations.SerializedName; import lombok.Getter; import lombok.Setter; import lombok.extern.log4j.Log4j2; -import static org.apache.doris.job.common.LoadConstants.STARTUP_MODE; -import static org.apache.doris.job.util.StreamingJobUtils.createMetaTableIfNotExist; -import static org.apache.doris.job.util.StreamingJobUtils.insertSplitsToMeta; -import org.apache.http.client.methods.HttpPost; -import org.apache.http.entity.StringEntity; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.HttpClientBuilder; -import org.apache.http.util.EntityUtils; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.collections4.MapUtils; +import java.io.IOException; +import java.util.ArrayList; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.stream.Collectors; @Getter @@ -39,6 +64,7 @@ @Log4j2 public class JdbcSourceOffsetProvider implements SourceOffsetProvider { public static final String BINLOG_SPLIT_ID = "binlog-split"; + public static final String SPLIT_ID = "splitId"; private static final ObjectMapper objectMapper = new ObjectMapper(); private Long jobId; private DataSourceType sourceType; @@ -50,6 +76,12 @@ public class JdbcSourceOffsetProvider implements SourceOffsetProvider { JdbcOffset currentOffset; Map endBinlogOffset; + @SerializedName("chw") + // tableID -> splitId -> chunk of highWatermark + Map>> chunkHighWatermarkMap; + @SerializedName("chw") + Map binlogOffsetPersist; + @Override public String getSourceType() { return "jdbc"; @@ -63,7 +95,7 @@ public Offset getNextOffset(StreamingJobProperties jobProps, Map SnapshotSplit snapshotSplit = remainingSplits.get(0); nextOffset.setSplit(snapshotSplit); return nextOffset; - }else if (currentOffset.getSplit().snapshotSplit()){ + } else if (currentOffset.getSplit().snapshotSplit()) { // snapshot to binlog BinlogSplit binlogSplit = new BinlogSplit(); binlogSplit.setSplitId(BINLOG_SPLIT_ID); @@ -78,89 +110,251 @@ public Offset getNextOffset(StreamingJobProperties jobProps, Map @Override public String getShowCurrentOffset() { + if (this.currentOffset != null) { + AbstractSourceSplit split = this.currentOffset.getSplit(); + if (split.snapshotSplit()) { + return new Gson().toJson(split); + } else { + BinlogSplit binlogSplit = (BinlogSplit) split; + return new Gson().toJson(binlogSplit.getStartingOffset()); + } + } return null; } @Override public String getShowMaxOffset() { + if (endBinlogOffset != null) { + return new Gson().toJson(endBinlogOffset); + } return null; } @Override public InsertIntoTableCommand rewriteTvfParams(InsertIntoTableCommand originCommand, Offset nextOffset) { - // not need + // todo: only for cdc tvf return null; } + /** + * + */ @Override public void updateOffset(Offset offset) { - + this.currentOffset = (JdbcOffset) offset; + AbstractSourceSplit split = currentOffset.getSplit(); + if (split.snapshotSplit()) { + SnapshotSplit snapshotSplit = (SnapshotSplit) split; + String splitId = split.getSplitId(); + remainingSplits.removeIf(v -> { + if (v.getSplitId().equals(splitId)) { + snapshotSplit.setTableId(v.getTableId()); + snapshotSplit.setSplitKey(v.getSplitKey()); + snapshotSplit.setSplitStart(v.getSplitStart()); + snapshotSplit.setSplitEnd(v.getSplitEnd()); + return true; + } + return false; + }); + finishedSplits.add(snapshotSplit); + chunkHighWatermarkMap.computeIfAbsent(snapshotSplit.getTableId(), k -> new HashMap<>()) + .put(snapshotSplit.getSplitId(), snapshotSplit.getHighWatermark()); + } else { + BinlogSplit binlogSplit = (BinlogSplit) split; + binlogOffsetPersist = new HashMap<>(binlogSplit.getStartingOffset()); + binlogOffsetPersist.put(SPLIT_ID, BINLOG_SPLIT_ID); + } } @Override public void fetchRemoteMeta(Map properties) throws Exception { - // todo: change to request be + Backend backend = StreamingJobUtils.selectBackend(jobId); Map params = buildBaseParams(); - String url = "http://127.0.0.1:9096/api/fetchEndOffset"; - // Prepare request body - String requestBody = GsonUtils.GSON.toJson(params); - - // Create HTTP POST request - HttpPost httpPost = new HttpPost(url); - StringEntity stringEntity = new StringEntity(requestBody, "UTF-8"); - stringEntity.setContentType("application/json"); - httpPost.setEntity(stringEntity); - // Set request headers - httpPost.setHeader("Content-Type", "application/json"); - - // Execute request - try (CloseableHttpClient client = HttpClientBuilder.create().build()) { - String response = client.execute(httpPost, httpResponse -> { - int statusCode = httpResponse.getStatusLine().getStatusCode(); - String responseBody = EntityUtils.toString(httpResponse.getEntity(), "UTF-8"); - if (statusCode != 200) { - throw new RuntimeException("Failed to get remote offset from CDC client, HTTP status code: " + statusCode); - } - return responseBody; - }); - + InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() + .setApi("/api/fetchEndOffset") + .setParams(GsonUtils.GSON.toJson(params)).build(); + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); + InternalService.PRequestCdcClientResult result = null; + try { + Future future = + BackendServiceProxy.getInstance().requestCdcClient(address, request); + result = future.get(); + TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + log.error("Failed to get end offset from backend, {}", result.getStatus().getErrorMsgs(0)); + throw new JobException( + "Failed to get end offset from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + + result.getResponse()); + } + String response = result.getResponse(); ResponseBody> responseObj = objectMapper.readValue( response, - new TypeReference>>() {} + new TypeReference>>() { + } ); endBinlogOffset = responseObj.getData(); - } catch (Exception ex) { - log.error("Get splits error: ", ex); - throw new JobException("Failed to request CDC client: " + ex.getMessage(), ex); + } catch (ExecutionException | InterruptedException | IOException ex) { + log.error("Get end offset error: ", ex); + throw new JobException(ex); } } @Override public boolean hasMoreDataToConsume() { - if(!remainingSplits.isEmpty()) { + if (!remainingSplits.isEmpty()) { return true; } if (currentOffset != null && !currentOffset.getSplit().snapshotSplit()) { BinlogSplit binlogSplit = (BinlogSplit) currentOffset.getSplit(); Map startingOffset = binlogSplit.getStartingOffset(); - // todo: should compare offset + try { + return compareOffset(startingOffset, endBinlogOffset); + } catch (Exception ex) { + log.info("Compare offset error: ", ex); + return false; + } } return true; } + private boolean compareOffset(Map offsetFirst, Map offsetSecond) + throws JobException { + Backend backend = StreamingJobUtils.selectBackend(jobId); + Map params = buildCompareOffsetParams(offsetFirst, offsetSecond); + InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() + .setApi("/api/compareOffset") + .setParams(GsonUtils.GSON.toJson(params)).build(); + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); + InternalService.PRequestCdcClientResult result = null; + try { + Future future = + BackendServiceProxy.getInstance().requestCdcClient(address, request); + result = future.get(); + TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + log.error("Failed to compare offset , {}", result.getStatus().getErrorMsgs(0)); + throw new JobException( + "Failed to compare offset ," + result.getStatus().getErrorMsgs(0) + ", response: " + + result.getResponse()); + } + String response = result.getResponse(); + ResponseBody responseObj = objectMapper.readValue( + response, + new TypeReference>() { + } + ); + return responseObj.getData() > 0; + } catch (ExecutionException | InterruptedException | IOException ex) { + log.error("Compare offset error: ", ex); + throw new JobException(ex); + } + } + @Override public Offset deserializeOffset(String offset) { - return null; + try { + // chunk is highWatermark, binlog is offset map + Map offsetMeta = objectMapper.readValue(offset, new TypeReference>() { + }); + String splitId = offsetMeta.remove(SPLIT_ID); + if (BINLOG_SPLIT_ID.equals(splitId)) { + BinlogSplit binlogSplit = new BinlogSplit(); + binlogSplit.setSplitId(splitId); + binlogSplit.setStartingOffset(offsetMeta); + return new JdbcOffset(binlogSplit); + } else { + SnapshotSplit snapshotSplit = new SnapshotSplit(); + snapshotSplit.setSplitId(splitId); + snapshotSplit.setHighWatermark(offsetMeta); + return new JdbcOffset(snapshotSplit); + } + } catch (JsonProcessingException e) { + log.warn("Failed to deserialize offset: {}", offset, e); + throw new RuntimeException(e); + } } @Override public Offset deserializeOffsetProperty(String offset) { + // todo: use for alter offset for job return null; } + /** + * Replay snapshot splits if needed + */ + @Override + public void replayIfNeed(StreamingInsertJob job) { + if (remainingSplits == null + && checkNeedSplitChunks(job.getSourceProperties())) { + try { + Map> snapshotSplits = StreamingJobUtils.restoreSplitsToJob(job.getJobId()); + String offsetProviderPersist = job.getOffsetProviderPersist(); + JdbcSourceOffsetProvider replayFromPersist = GsonUtils.GSON.fromJson(offsetProviderPersist, + JdbcSourceOffsetProvider.class); + this.chunkHighWatermarkMap = replayFromPersist.getChunkHighWatermarkMap(); + if (MapUtils.isNotEmpty(chunkHighWatermarkMap) && MapUtils.isNotEmpty(snapshotSplits)) { + replaySnapshotSplits(chunkHighWatermarkMap, snapshotSplits); + } + this.binlogOffsetPersist = replayFromPersist.getBinlogOffsetPersist(); + if (MapUtils.isNotEmpty(binlogOffsetPersist)) { + currentOffset = new JdbcOffset(); + currentOffset.setSplit(new BinlogSplit(BINLOG_SPLIT_ID, binlogOffsetPersist)); + } + } catch (Exception ex) { + log.error("Failed to restore splits for job {}", job.getJobId(), ex); + } + } + } + + /** + * Assign the HW value to the synchronized Split, + * and remove the Split from remainSplit and place it in finishedSplit. + */ + private void replaySnapshotSplits(Map>> chunkHighWatermarkMap, + Map> snapshotSplits) { + if (this.finishedSplits == null) { + this.finishedSplits = new ArrayList<>(); + } + + for (Map.Entry>> entry : chunkHighWatermarkMap.entrySet()) { + String tableId = entry.getKey(); + Map> splitIdToHighWatermark = entry.getValue(); + if (MapUtils.isEmpty(splitIdToHighWatermark)) { + continue; + } + + List tableSplits = snapshotSplits.get(tableId); + if (CollectionUtils.isEmpty(tableSplits)) { + continue; + } + + for (Iterator iterator = tableSplits.iterator(); iterator.hasNext(); ) { + SnapshotSplit split = iterator.next(); + String splitId = split.getSplitId(); + + Map highWatermark = splitIdToHighWatermark.get(splitId); + if (highWatermark != null) { + split.setHighWatermark(highWatermark); + finishedSplits.add(split); + iterator.remove(); + } + } + } + + this.remainingSplits = snapshotSplits.values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); + } + + @Override + public String getPersistInfo() { + return GsonUtils.GSON.toJson(this); + } + public void splitChunks(List createTbls) throws JobException { // todo: When splitting takes a long time, it needs to be changed to asynchronous. - if(checkNeedSplitChunks(sourceProperties)) { + if (checkNeedSplitChunks(sourceProperties)) { Map> tableSplits = new HashMap<>(); for (String tbl : createTbls) { List snapshotSplits = requestTableSplits(tbl); @@ -176,8 +370,8 @@ public void splitChunks(List createTbls) throws JobException { private void saveChunkMeta(Map> tableSplits) throws JobException { try { - createMetaTableIfNotExist(); - insertSplitsToMeta(getJobId(), tableSplits); + StreamingJobUtils.createMetaTableIfNotExist(); + StreamingJobUtils.insertSplitsToMeta(getJobId(), tableSplits); } catch (Exception e) { log.warn("save chunk meta error: ", e); throw new JobException(e.getMessage()); @@ -185,44 +379,7 @@ private void saveChunkMeta(Map> tableSplits) throws } private List requestTableSplits(String table) throws JobException { - Map params = buildSplitParams(table); - String url = "http://127.0.0.1:9096/api/fetchSplits"; - // Prepare request body - String requestBody = GsonUtils.GSON.toJson(params); - - // Create HTTP POST request - HttpPost httpPost = new HttpPost(url); - StringEntity stringEntity = new StringEntity(requestBody, "UTF-8"); - stringEntity.setContentType("application/json"); - httpPost.setEntity(stringEntity); - // Set request headers - httpPost.setHeader("Content-Type", "application/json"); - - // Execute request - try (CloseableHttpClient client = HttpClientBuilder.create().build()) { - String response = client.execute(httpPost, httpResponse -> { - int statusCode = httpResponse.getStatusLine().getStatusCode(); - String responseBody = EntityUtils.toString(httpResponse.getEntity(), "UTF-8"); - if (statusCode != 200) { - throw new RuntimeException("Failed to get split from CDC client, HTTP status code: " + statusCode); - } - return responseBody; - }); - - ResponseBody> responseObj = objectMapper.readValue( - response, - new TypeReference>>() {} - ); - List splits = responseObj.getData(); - return splits; - - } catch (Exception ex) { - log.error("Get splits error: ", ex); - throw new JobException("Failed to request CDC client: " + ex.getMessage(), ex); - } - - /** - Backend backend = selectBackend(jobId); + Backend backend = StreamingJobUtils.selectBackend(jobId); Map params = buildSplitParams(table); InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() .setApi("/api/fetchSplits") @@ -236,15 +393,22 @@ private List requestTableSplits(String table) throws JobException TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); if (code != TStatusCode.OK) { log.error("Failed to get split from backend, {}", result.getStatus().getErrorMsgs(0)); - throw new JobException("Failed to get split from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + result.getResponse()); + throw new JobException( + "Failed to get split from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + + result.getResponse()); } - } catch (ExecutionException | InterruptedException ex) { + String response = result.getResponse(); + ResponseBody> responseObj = objectMapper.readValue( + response, + new TypeReference>>() { + } + ); + List splits = responseObj.getData(); + return splits; + } catch (ExecutionException | InterruptedException | IOException ex) { log.error("Get splits error: ", ex); throw new JobException(ex); } - log.info("========fetch cdc split {}", result.getResponse()); - return ""; - **/ } private Map buildBaseParams() { @@ -255,35 +419,22 @@ private Map buildBaseParams() { return params; } - private Map buildSplitParams(String table) { + private Map buildCompareOffsetParams(Map offsetFirst, + Map offsetSecond) { Map params = buildBaseParams(); - params.put("snapshotTable", table); + params.put("offsetFirst", offsetFirst); + params.put("offsetSecond", offsetSecond); return params; } - public static Backend selectBackend(Long jobId) throws JobException { - Backend backend = null; - BeSelectionPolicy policy = null; - - policy = new BeSelectionPolicy.Builder() - .setEnableRoundRobin(true) - .needLoadAvailable().build(); - List backendIds; - backendIds = Env.getCurrentSystemInfo().selectBackendIdsByPolicy(policy, 1); - if (backendIds.isEmpty()) { - throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); - } - // jobid % backendSize - long index = backendIds.get(jobId.intValue() % backendIds.size()); - backend = Env.getCurrentSystemInfo().getBackend(index); - if (backend == null) { - throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); - } - return backend; + private Map buildSplitParams(String table) { + Map params = buildBaseParams(); + params.put("snapshotTable", table); + return params; } private boolean checkNeedSplitChunks(Map sourceProperties) { - String startMode = sourceProperties.get(STARTUP_MODE); + String startMode = sourceProperties.get(LoadConstants.STARTUP_MODE); if (startMode == null) { return false; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java index c977876030662e..a4cf3a01285786 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java @@ -17,11 +17,11 @@ package org.apache.doris.job.offset.jdbc.split; -import com.fasterxml.jackson.databind.ObjectMapper; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.Setter; + import java.io.Serializable; @Getter diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java index 94e4e6e57ab91e..bd082be2082733 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java @@ -17,13 +17,12 @@ package org.apache.doris.job.offset.jdbc.split; -import org.apache.doris.persist.gson.GsonUtils; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.Setter; + import java.util.List; import java.util.Map; @@ -33,11 +32,19 @@ @NoArgsConstructor public class BinlogSplit extends AbstractSourceSplit { private static final long serialVersionUID = 1L; + private Map startingOffset; private Map endingOffset; // binlog split meta, first binlog split requires private List finishedSplits; + public BinlogSplit( + String splitId, + Map startingOffset) { + this.splitId = splitId; + this.startingOffset = startingOffset; + } + @Override public String toString() { return new Gson().toJson(this); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java index 271be64e73a26f..eb5f992c345535 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java @@ -17,20 +17,15 @@ package org.apache.doris.job.offset.jdbc.split; -import org.apache.doris.persist.gson.GsonUtils; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; import com.google.gson.Gson; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.Setter; -import java.util.Arrays; + import java.util.List; import java.util.Map; -import java.util.Optional; @Getter @Setter @@ -60,50 +55,6 @@ public SnapshotSplit( this.highWatermark = highWatermark; } - public static SnapshotSplit fromMap(Map map) throws JsonProcessingException { - if (map == null || map.isEmpty()) { - return null; - } - - SnapshotSplit split = new SnapshotSplit(); - String splitId = map.get("splitId"); - String tableId = map.get("tableId"); - String splitKeyStr = map.get("splitKey"); - Preconditions.checkNotNull(splitKeyStr, "splitKey must not be null"); - List splitKey = - objectMapper.readValue(splitKeyStr, new TypeReference>() {}); - - split.setSplitId(splitId); - split.setTableId(tableId); - split.setSplitKey(splitKey); - - String splitStartStr = map.get("splitStart"); - if (splitStartStr != null) { - Object[] splitStart = objectMapper.readValue(splitStartStr, Object[].class); - split.setSplitStart(splitStart); - } - - String splitEndStr = map.get("splitEnd"); - if (splitEndStr != null) { - Object[] splitEnd = objectMapper.readValue(splitEndStr, Object[].class); - split.setSplitEnd(splitEnd); - } - - String highWatermarkStr = map.get("highWatermark"); - if (highWatermarkStr != null) { - Map highWatermark = - objectMapper.readValue( - highWatermarkStr, new TypeReference>() {}); - split.setHighWatermark(highWatermark); - } - - return split; - } - - public static String getOrEmptyArray(Map map, String key) { - return Optional.ofNullable(map.get(key)).orElse("[]"); - } - @Override public String toString() { return new Gson().toJson(this); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java index b19839ef646ad4..91d5fb1a658737 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java @@ -27,7 +27,6 @@ import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.streaming.AbstractStreamingTask; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; -import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; import lombok.extern.log4j.Log4j2; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java index 33589827361a06..2785c7b9b00e50 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.job.util; import org.apache.doris.analysis.UserIdentity; @@ -25,13 +42,15 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.statistics.ResultRow; +import org.apache.doris.system.Backend; +import org.apache.doris.system.BeSelectionPolicy; +import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TUniqueId; import com.fasterxml.jackson.databind.ObjectMapper; import lombok.extern.log4j.Log4j2; import org.apache.commons.text.StringSubstitutor; -import static org.apache.doris.job.common.LoadConstants.DRIVER_URL; -import static org.apache.doris.job.common.LoadConstants.*; import java.util.ArrayList; import java.util.Arrays; @@ -57,13 +76,16 @@ public class StreamingJobUtils { + "UNIQUE KEY(id)\n" + "DISTRIBUTED BY HASH(id)\n" + "BUCKETS 1\n" - + "PROPERTIES ('replication_num' = '1')"; //todo: modify replication num like statistic sys tbl + + "PROPERTIES ('replication_num' = '1')"; // todo: modify replication num like statistic sys tbl private static final String BATCH_INSERT_INTO_META_TABLE_TEMPLATE = "INSERT INTO " + FULL_QUALIFIED_META_TBL_NAME + " values"; private static final String INSERT_INTO_META_TABLE_TEMPLATE = "('${id}', '${job_id}', '${table_name}', '${chunk_list}')"; + private static final String SELECT_SPLITS_TABLE_TEMPLATE = + "SELECT table_name, chunk_list from " + FULL_QUALIFIED_META_TBL_NAME + " WHERE job_id='%s'"; + private static final ObjectMapper objectMapper = new ObjectMapper(); public static void createMetaTableIfNotExist() throws Exception { @@ -82,16 +104,34 @@ public static void createMetaTableIfNotExist() throws Exception { // double check t = database.getTableNullable(INTERNAL_STREAMING_JOB_META_TABLE_NAME); - if (t == null){ + if (t == null) { throw new JobException(String.format("Table %s doesn't exist", FULL_QUALIFIED_META_TBL_NAME)); } } + public static Map> restoreSplitsToJob(Long jobId) throws Exception { + List resultRows = new ArrayList<>(); + String sql = String.format(SELECT_SPLITS_TABLE_TEMPLATE, jobId); + try (AutoCloseConnectContext r = buildConnectContext()) { + StmtExecutor stmtExecutor = new StmtExecutor(r.connectContext, sql); + resultRows = stmtExecutor.executeInternalQuery(); + } + + Map> tableSplits = new HashMap<>(); + for (ResultRow row : resultRows) { + String tableName = row.get(0); + String chunkListStr = row.get(1); + List splits = Arrays.asList(objectMapper.readValue(chunkListStr, SnapshotSplit[].class)); + tableSplits.put(tableName, splits); + } + return tableSplits; + } + public static void insertSplitsToMeta(Long jobId, Map> tableSplits) throws Exception { List values = new ArrayList<>(); for (Map.Entry> entry : tableSplits.entrySet()) { Map params = new HashMap<>(); - params.put("id", UUID.randomUUID().toString().replace("-","")); + params.put("id", UUID.randomUUID().toString().replace("-", "")); params.put("job_id", jobId + ""); params.put("table_name", entry.getKey()); params.put("chunk_list", objectMapper.writeValueAsString(entry.getValue())); @@ -142,14 +182,15 @@ private static AutoCloseConnectContext buildConnectContext() { return new AutoCloseConnectContext(connectContext); } - private static JdbcClient getJdbcClient(DataSourceType sourceType, Map properties) throws JobException { + private static JdbcClient getJdbcClient(DataSourceType sourceType, Map properties) + throws JobException { JdbcClientConfig config = new JdbcClientConfig(); config.setCatalog(sourceType.name()); - config.setUser(properties.get(USER)); - config.setPassword(properties.get(PASSWORD)); - config.setDriverClass(properties.get(DRIVER_CLASS)); - config.setDriverUrl(properties.get(DRIVER_URL)); - config.setJdbcUrl(properties.get(JDBC_URL)); + config.setUser(properties.get(LoadConstants.USER)); + config.setPassword(properties.get(LoadConstants.PASSWORD)); + config.setDriverClass(properties.get(LoadConstants.DRIVER_CLASS)); + config.setDriverUrl(properties.get(LoadConstants.DRIVER_URL)); + config.setJdbcUrl(properties.get(LoadConstants.JDBC_URL)); switch (sourceType) { case MYSQL: JdbcClient client = JdbcMySQLClient.createJdbcClient(config); @@ -159,17 +200,39 @@ private static JdbcClient getJdbcClient(DataSourceType sourceType, Map generateCreateTableInfos(String targetDb, DataSourceType sourceType, Map properties, Map targetProperties) + public static Backend selectBackend(Long jobId) throws JobException { + Backend backend = null; + BeSelectionPolicy policy = null; + + policy = new BeSelectionPolicy.Builder() + .setEnableRoundRobin(true) + .needLoadAvailable().build(); + List backendIds; + backendIds = Env.getCurrentSystemInfo().selectBackendIdsByPolicy(policy, -1); + if (backendIds.isEmpty()) { + throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); + } + // jobid % backendSize + long index = backendIds.get(jobId.intValue() % backendIds.size()); + backend = Env.getCurrentSystemInfo().getBackend(index); + if (backend == null) { + throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); + } + return backend; + } + + public static List generateCreateTableInfos(String targetDb, DataSourceType sourceType, + Map properties, Map targetProperties) throws JobException { List createtblInfos = new ArrayList<>(); - String includeTables = properties.get(INCLUDE_TABLES); - String excludeTables = properties.get(EXCLUDE_TABLES); + String includeTables = properties.get(LoadConstants.INCLUDE_TABLES); + String excludeTables = properties.get(LoadConstants.EXCLUDE_TABLES); List includeTablesList = new ArrayList<>(); if (includeTables != null) { includeTablesList = Arrays.asList(includeTables.split(",")); } - String database = properties.get(DATABASE); + String database = properties.get(LoadConstants.DATABASE); JdbcClient jdbcClient = getJdbcClient(sourceType, properties); List tablesNameList = jdbcClient.getTablesNameList(database); if (tablesNameList.isEmpty()) { @@ -183,7 +246,7 @@ public static List generateCreateTableInfos(String targetDb, Da continue; } - if(excludeTables != null && excludeTables.contains(table)) { + if (excludeTables != null && excludeTables.contains(table)) { log.info("Skip table {} in database {} as it in exclude_tables {}", table, database, excludeTables); continue; @@ -195,7 +258,8 @@ public static List generateCreateTableInfos(String targetDb, Da .collect(Collectors.toList()); if (primaryKeys.isEmpty()) { primaryKeys.add(columns.get(0).getName()); - log.info("table {} no primary key, use first column {} to primary key", table, columns.get(0).getName()); + log.info("table {} no primary key, use first column {} to primary key", table, + columns.get(0).getName()); } // Convert Column to ColumnDefinition List columnDefinitions = columns.stream().map(col -> { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index e91e9b191d8294..c34272f8743171 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -1186,7 +1186,7 @@ public LogicalPlan visitCreateScheduledJob(DorisParser.CreateScheduledJobContext ? Maps.newHashMap(visitPropertyItemList(ctx.targetProperties)) : Maps.newHashMap(); CreateJobInfo createJobInfo = new CreateJobInfo(label, atTime, interval, intervalUnit, startTime, endsTime, immediateStartOptional, comment, executeSql, ctx.STREAMING() != null, - jobProperties,sourceType, targetDb, sourceProperties, targetProperties); + jobProperties, sourceType, targetDb, sourceProperties, targetProperties); return new CreateJobCommand(createJobInfo); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java index c1384efb9dbf2e..327f98d4c1ac8d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java @@ -157,8 +157,9 @@ public AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws UserExcepti if (streamingJob) { if (sourceType.isPresent()) { DataSourceType dataSourceType = DataSourceType.valueOf(sourceType.get()); - return analyzeAndCreateFromSourceJob(dbName, jobExecutionConfiguration, jobProperties, targetDb, dataSourceType, sourceProperties, targetProperties); - }else { + return analyzeAndCreateFromSourceJob(dbName, jobExecutionConfiguration, + jobProperties, targetDb, dataSourceType, sourceProperties, targetProperties); + } else { return analyzeAndCreateStreamingInsertJob(executeSql, dbName, jobExecutionConfiguration, jobProperties); } } else { @@ -257,8 +258,12 @@ protected void checkAuth() throws AnalysisException { * @throws UserException if there is an error during SQL analysis or job creation */ private AbstractJob analyzeAndCreateFromSourceJob(String currentDbName, - JobExecutionConfiguration jobExecutionConfiguration, Map jobProperties, String targetDb, - DataSourceType dataSourceType, Map sourceProperties, Map targetProperties) throws UserException { + JobExecutionConfiguration jobExecutionConfiguration, + Map jobProperties, + String targetDb, + DataSourceType dataSourceType, + Map sourceProperties, + Map targetProperties) throws UserException { Optional db = Env.getCurrentEnv().getInternalCatalog().getDb(targetDb); if (!db.isPresent()) { throw new AnalysisException("Target database " + targetDb + " does not exist"); @@ -274,17 +279,6 @@ private AbstractJob analyzeAndCreateFromSourceJob(String currentDbName, jobProperties, targetDb, dataSourceType, sourceProperties, targetProperties); } - - private AbstractJob analyzeAndCreateJob(String sql, String currentDbName, - JobExecutionConfiguration jobExecutionConfiguration, - Map properties) throws UserException { - if (jobExecutionConfiguration.getExecuteType().equals(JobExecuteType.STREAMING)) { - return analyzeAndCreateStreamingInsertJob(sql, currentDbName, jobExecutionConfiguration, properties); - } else { - return analyzeAndCreateInsertJob(sql, currentDbName, jobExecutionConfiguration); - } - } - /** * Analyzes the provided SQL statement and creates an appropriate job based on the parsed logical plan. * Currently, only "InsertIntoTableCommand" is supported for job creation. diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java index 8b245f2e91b922..eff777c98d1157 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java @@ -214,11 +214,6 @@ public Future requestCdcClient( return stub.requestCdcClient(request); } - public Future executeCdcScanCommit( - InternalService.PRequestCdcClientRequest request) { - return stub.executeCdcScanCommit(request); - } - public void shutdown() { ConnectivityState state = channel.getState(false); LOG.warn("shut down backend service client: {}, channel state: {}", address, state); diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java index f7ce54e5ee82fb..68fa927f0f2fe5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java @@ -588,7 +588,8 @@ public Future abortDictionaryAs return null; } - public Future requestCdcClient(TNetworkAddress address, InternalService.PRequestCdcClientRequest request) { + public Future requestCdcClient(TNetworkAddress address, + InternalService.PRequestCdcClientRequest request) { try { final BackendServiceClient client = getProxy(address); return client.requestCdcClient(request); @@ -597,14 +598,4 @@ public Future requestCdcClient(TNetwork } return null; } - - public Future executeCdcScanCommit(TNetworkAddress address, InternalService.PRequestCdcClientRequest request) { - try { - final BackendServiceClient client = getProxy(address); - return client.executeCdcScanCommit(request); - } catch (Throwable e) { - LOG.warn("execute cdc scan commit failed, address={}:{}", address.getHostname(), address.getPort(), e); - } - return null; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index 84764b849aa8ee..94a978f29a9408 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -69,7 +69,6 @@ import org.apache.doris.job.common.JobType; import org.apache.doris.job.extensions.insert.streaming.AbstractStreamingTask; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; -import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; import org.apache.doris.job.extensions.mtmv.MTMVJob; import org.apache.doris.job.task.AbstractTask; import org.apache.doris.mtmv.MTMVPartitionUtil; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index b1f7754655fcf5..51a193a7fc77ff 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -17,15 +17,17 @@ package org.apache.doris.cdcclient.controller; +import jakarta.servlet.http.HttpServletRequest; +import java.util.List; import org.apache.doris.cdcclient.common.Env; import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.cdcclient.model.request.CompareOffsetReq; import org.apache.doris.cdcclient.model.request.FetchRecordReq; import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; import org.apache.doris.cdcclient.model.request.WriteRecordReq; import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; import org.apache.doris.cdcclient.service.PipelineCoordinator; import org.apache.doris.cdcclient.source.reader.SourceReader; -import jakarta.servlet.http.HttpServletRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -34,7 +36,6 @@ import org.springframework.web.bind.annotation.RequestMapping; import org.springframework.web.bind.annotation.RequestMethod; import org.springframework.web.bind.annotation.RestController; -import java.util.List; @RestController public class ClientController { @@ -69,8 +70,8 @@ public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { /** Fetch records from source reader and Write records to backend */ @RequestMapping(path = "/api/writeRecords", method = RequestMethod.POST) - public Object writeRecord(@RequestBody WriteRecordReq recordReq, HttpServletRequest request) { - pipelineCoordinator.writeRecordsAsync(recordReq, parseToken(request)); + public Object writeRecord(@RequestBody WriteRecordReq recordReq) { + pipelineCoordinator.writeRecordsAsync(recordReq); return ResponseEntityBuilder.ok("Request accepted, processing asynchronously"); } @@ -85,6 +86,15 @@ public Object fetchEndOffset(@RequestBody JobConfig jobConfig) { return ResponseEntityBuilder.ok(reader.getEndOffset(jobConfig)); } + /** compare datasource Binlog Offset */ + @RequestMapping(path = "/api/compareOffset", method = RequestMethod.POST) + public Object compareOffset(@RequestBody CompareOffsetReq compareOffsetReq) { + SourceReader reader = Env.getCurrentEnv().getReader(compareOffsetReq); + return ResponseEntityBuilder.ok( + reader.compareOffset( + compareOffsetReq.getOffsetFirst(), compareOffsetReq.getOffsetSecond())); + } + @RequestMapping(path = "/api/close/{jobId}", method = RequestMethod.POST) public Object close(@PathVariable long jobId) { Env env = Env.getCurrentEnv(); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java new file mode 100644 index 00000000000000..d5eb74834b53a3 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java @@ -0,0 +1,30 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model.request; + +import java.util.Map; +import lombok.Getter; +import lombok.Setter; +import org.apache.doris.cdcclient.model.JobConfig; + +@Getter +@Setter +public class CompareOffsetReq extends JobConfig { + private Map offsetFirst; + private Map offsetSecond; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java index 0b92444095affb..79c34028be02a1 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java @@ -25,7 +25,9 @@ public class WriteRecordReq extends JobBaseRecordReq { private long maxInterval; private String targetDb; - private String labelName; + private String token; + private String frontendAddress; + private String taskId; @Override public boolean isReload() { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 1113bf1aa43473..68af2d9856a278 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -78,24 +78,27 @@ public PipelineCoordinator() { new ThreadPoolExecutor.AbortPolicy()); } - public CompletableFuture writeRecordsAsync(WriteRecordReq writeRecordReq, String token) { - Preconditions.checkNotNull(token, "doris token must not be null"); - Preconditions.checkNotNull(writeRecordReq.getLabelName(), "labelName must not be null"); + public CompletableFuture writeRecordsAsync(WriteRecordReq writeRecordReq) { + Preconditions.checkNotNull(writeRecordReq.getToken(), "token must not be null"); + Preconditions.checkNotNull(writeRecordReq.getTaskId(), "taskId must not be null"); Preconditions.checkNotNull(writeRecordReq.getTargetDb(), "targetDb must not be null"); return CompletableFuture.runAsync( () -> { try { LOG.info( - "Start processing async write record, labelName={}", - writeRecordReq.getLabelName()); - writeRecords(writeRecordReq, token); + "Start processing async write record, jobId={} taskId={}", + writeRecordReq.getJobId(), + writeRecordReq.getTaskId()); + writeRecords(writeRecordReq); LOG.info( - "Successfully processed async write record, labelName={}", - writeRecordReq.getLabelName()); + "Successfully processed async write record, jobId={} taskId={}", + writeRecordReq.getJobId(), + writeRecordReq.getTaskId()); } catch (Exception ex) { LOG.error( - "Failed to process async write record, labelName={}", - writeRecordReq.getLabelName(), + "Failed to process async write record, jobId={} taskId={}", + writeRecordReq.getJobId(), + writeRecordReq.getTaskId(), ex); } }, @@ -103,7 +106,7 @@ public CompletableFuture writeRecordsAsync(WriteRecordReq writeRecordReq, } /** Read data from SourceReader and write it to Doris, while returning meta information. */ - public void writeRecords(WriteRecordReq writeRecordReq, String token) throws Exception { + public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordReq); DorisBatchStreamLoad batchStreamLoad = null; Map metaResponse = new HashMap<>(); @@ -112,8 +115,9 @@ public void writeRecords(WriteRecordReq writeRecordReq, String token) throws Exc batchStreamLoad = getOrCreateBatchStreamLoad( writeRecordReq.getJobId(), writeRecordReq.getTargetDb()); - batchStreamLoad.setCurrentLabel(writeRecordReq.getLabelName()); - batchStreamLoad.setToken(token); + batchStreamLoad.setCurrentTaskId(writeRecordReq.getTaskId()); + batchStreamLoad.setFrontendAddress(writeRecordReq.getFrontendAddress()); + batchStreamLoad.setToken(writeRecordReq.getToken()); boolean readBinlog = readResult.isReadBinlog(); boolean pureBinlogPhase = readResult.isPureBinlogPhase(); @@ -174,11 +178,13 @@ public void writeRecords(WriteRecordReq writeRecordReq, String token) throws Exc } metaResponse = offsetRes; } - batchStreamLoad.commitTransaction(metaResponse); + // request fe api + batchStreamLoad.commitOffset(metaResponse); + // batchStreamLoad.commitTransaction(metaResponse); } finally { sourceReader.finishSplitRecords(); if (batchStreamLoad != null) { - batchStreamLoad.resetLabel(); + batchStreamLoad.resetTaskId(); } } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java index a2f9db236327f7..63023eced4bf3f 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java @@ -17,7 +17,19 @@ package org.apache.doris.cdcclient.sink; +import org.apache.doris.cdcclient.common.Env; +import org.apache.doris.cdcclient.exception.StreamLoadException; +import org.apache.doris.cdcclient.utils.HttpUtil; import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.Setter; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.util.EntityUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -43,18 +55,6 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import lombok.Setter; -import org.apache.commons.lang3.StringUtils; -import org.apache.doris.cdcclient.common.Env; -import org.apache.doris.cdcclient.exception.StreamLoadException; -import org.apache.doris.cdcclient.utils.HttpUtil; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.entity.StringEntity; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.util.EntityUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** async stream load. */ public class DorisBatchStreamLoad implements Serializable { @@ -68,8 +68,9 @@ public class DorisBatchStreamLoad implements Serializable { private final int RETRY = 3; private final byte[] lineDelimiter = "\n".getBytes(); private static final String LOAD_URL_PATTERN = "http://%s/api/%s/%s/_stream_load"; - private static final String COMMIT_URL_PATTERN = "http://%s/api/%s/_stream_load_2pc"; + private static final String COMMIT_URL_PATTERN = "http://%s/api/streaming/commit_offset"; private String hostPort; + @Setter private String frontendAddress; private Map bufferMap = new ConcurrentHashMap<>(); private ExecutorService loadExecutorService; private LoadAsyncExecutor loadAsyncExecutor; @@ -82,8 +83,7 @@ public class DorisBatchStreamLoad implements Serializable { private final Lock lock = new ReentrantLock(); private final Condition block = lock.newCondition(); private final Map bufferMapLock = new ConcurrentHashMap<>(); - @Setter private String currentLabel; - private List successSubLabels = new ArrayList<>(); + @Setter private String currentTaskId; private String targetDb; private long jobId; @Setter private String token; @@ -197,7 +197,7 @@ private synchronized void flushBuffer(String bufferKey) { LOG.info("buffer key is not exist {}, skipped", bufferKey); return; } - buffer.setLabelName(UUID.randomUUID().toString()); + buffer.setLabelName(UUID.randomUUID().toString().replace("-", "")); LOG.debug("flush buffer for key {} with label {}", bufferKey, buffer.getLabelName()); putRecordToFlushQueue(buffer); } @@ -354,13 +354,12 @@ public void load(String label, BatchRecordBuffer buffer) throws IOException { BatchBufferHttpEntity entity = new BatchBufferHttpEntity(buffer); HttpPutBuilder putBuilder = new HttpPutBuilder(); String loadUrl = String.format(LOAD_URL_PATTERN, hostPort, targetDb, buffer.getTable()); + String finalLabel = String.format("%s_%s_%s", jobId, currentTaskId, label); putBuilder .setUrl(loadUrl) .addTokenAuth(token) - .setLabel(currentLabel) - .setSubLabel(label) + .setLabel(finalLabel) .formatJson() - .enable2PC() .addCommonHeader() .setEntity(entity) .addHiddenColumns(true) @@ -379,7 +378,6 @@ public void load(String label, BatchRecordBuffer buffer) throws IOException { RespContent respContent = OBJECT_MAPPER.readValue(loadResult, RespContent.class); if (DORIS_SUCCESS_STATUS.contains(respContent.getStatus())) { - successSubLabels.add(label); long cacheByteBeforeFlush = currentCacheBytes.getAndAdd(-respContent.getLoadBytes()); LOG.info( @@ -458,33 +456,29 @@ public Thread newThread(Runnable r) { } } - public void resetLabel() { - this.currentLabel = null; - this.successSubLabels.clear(); + public void resetTaskId() { + this.currentTaskId = null; } - /** commit transaction. */ - public void commitTransaction(Map meta) { + /** commit offfset to frontends. */ + public void commitOffset(Map meta) { try { - String url = String.format(COMMIT_URL_PATTERN, hostPort, targetDb); - - Map attachment = new HashMap<>(); - attachment.put("offset", OBJECT_MAPPER.writeValueAsString(meta)); + String url = String.format(COMMIT_URL_PATTERN, frontendAddress, targetDb); Map commitParams = new HashMap<>(); - commitParams.put("attachment", attachment); - commitParams.put("sub_labels", successSubLabels); + commitParams.put("offset", OBJECT_MAPPER.writeValueAsString(meta)); + commitParams.put("jobId", jobId); + commitParams.put("taskId", currentTaskId); HttpPutBuilder builder = new HttpPutBuilder() .addCommonHeader() .addTokenAuth(token) - .setLabel(currentLabel) .setUrl(url) .commit() .setEntity( new StringEntity( OBJECT_MAPPER.writeValueAsString(commitParams))); - LOG.info("commit transaction with label: {}", currentLabel); + LOG.info("commit offset for jobId {} taskId {}", jobId, currentTaskId); Throwable resEx = null; int retry = 0; while (retry <= RETRY) { @@ -498,24 +492,19 @@ public void commitTransaction(Map meta) { : ""; LOG.info("commit result {}", responseBody); if (statusCode == 200) { - LOG.info("commit transaction success, label: {}", currentLabel); + LOG.info("commit offset for jobId {} taskId {}", jobId, currentTaskId); return; } LOG.error( - "commit transaction failed with {}, reason {}, to retry", + "commit offset failed with {}, reason {}, to retry", hostPort, reason); if (retry == RETRY) { - resEx = - new StreamLoadException( - "commit transaction failed with: " + reason); + resEx = new StreamLoadException("commit offset failed with: " + reason); } } catch (Exception ex) { resEx = ex; - LOG.error( - "commit transaction error with {}, to retry, cause by", - hostPort, - ex); + LOG.error("commit offset error with {}, to retry, cause by", hostPort, ex); } } retry++; @@ -531,13 +520,13 @@ public void commitTransaction(Map meta) { if (retry > RETRY) { throw new StreamLoadException( - "commit transaction error: " + "commit offset error: " + (resEx != null ? resEx.getMessage() : "unknown error"), resEx); } } catch (Exception ex) { - LOG.error("Failed to commit transaction, jobId={}", jobId, ex); - throw new StreamLoadException("Failed to commit transaction", ex); + LOG.error("Failed to commit offset, jobId={}", jobId, ex); + throw new StreamLoadException("Failed to commit offset", ex); } } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java index 39b5699b62d07d..337b1e197df7ea 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java @@ -113,13 +113,6 @@ public HttpPutBuilder setLabel(String label) { return this; } - public HttpPutBuilder setSubLabel(String subLabel) { - if (subLabel != null) { - header.put("sub_label", subLabel); - } - return this; - } - public String getLabel() { return header.get("label"); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index 1d820877a42ab6..05cb5f6e33f27a 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -70,4 +70,6 @@ default SplitReadResult readSplitRecords(JobBaseRecordReq bas void finishSplitRecords(); Map getEndOffset(JobConfig jobConfig); + + int compareOffset(Map offsetFirst, Map offsetSecond); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index c2790a7e483a39..d561896a185ce8 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -613,6 +613,13 @@ public Map getEndOffset(JobConfig jobConfig) { } } + @Override + public int compareOffset(Map offsetFirst, Map offsetSecond) { + BinlogOffset binlogOffset1 = new BinlogOffset(offsetFirst); + BinlogOffset binlogOffset2 = new BinlogOffset(offsetSecond); + return binlogOffset1.compareTo(binlogOffset2); + } + private Map getTableSchemas(JobConfig config) { Map schemas = jobRuntimeContext.getTableSchemas(); if (schemas == null) { From 06c8ae7bbfe43492413650dc6a4b5609334d1a3e Mon Sep 17 00:00:00 2001 From: JNSimba <676366545@qq.com> Date: Sat, 6 Dec 2025 14:46:03 +0800 Subject: [PATCH 11/27] add build script --- build.sh | 29 ++++++++++++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/build.sh b/build.sh index 41041ae3f66696..35c20d6299d95b 100755 --- a/build.sh +++ b/build.sh @@ -142,6 +142,7 @@ if ! OPTS="$(getopt \ -l 'spark-dpp' \ -l 'hive-udf' \ -l 'be-java-extensions' \ + -l 'be-cdc-client' \ -l 'be-extension-ignore:' \ -l 'clean' \ -l 'coverage' \ @@ -165,6 +166,7 @@ BUILD_INDEX_TOOL='OFF' BUILD_BENCHMARK='OFF' BUILD_TASK_EXECUTOR_SIMULATOR='OFF' BUILD_BE_JAVA_EXTENSIONS=0 +BUILD_BE_CDC_CLIENT=0 BUILD_HIVE_UDF=0 CLEAN=0 HELP=0 @@ -187,6 +189,7 @@ if [[ "$#" == 1 ]]; then BUILD_BENCHMARK='OFF' BUILD_HIVE_UDF=1 BUILD_BE_JAVA_EXTENSIONS=1 + BUILD_BE_CDC_CLIENT=1 CLEAN=0 else while true; do @@ -200,6 +203,7 @@ else --be) BUILD_BE=1 BUILD_BE_JAVA_EXTENSIONS=1 + BUILD_BE_CDC_CLIENT=1 shift ;; --cloud) @@ -245,6 +249,10 @@ else BUILD_BE_JAVA_EXTENSIONS=1 shift ;; + --be-cdc-client) + BUILD_BE_CDC_CLIENT=1 + shift + ;; --clean) CLEAN=1 shift @@ -296,6 +304,7 @@ else BUILD_TASK_EXECUTOR_SIMULATOR='OFF' BUILD_HIVE_UDF=1 BUILD_BE_JAVA_EXTENSIONS=1 + BUILD_BE_CDC_CLIENT=1 CLEAN=0 fi fi @@ -426,6 +435,14 @@ if [[ -n "${DISABLE_BE_JAVA_EXTENSIONS}" ]]; then fi fi +if [[ -n "${DISABLE_BE_CDC_CLIENT}" ]]; then + if [[ "${DISABLE_BE_CDC_CLIENT}" == "ON" ]]; then + BUILD_BE_CDC_CLIENT=0 + else + BUILD_BE_CDC_CLIENT=1 + fi +fi + if [[ -n "${DISABLE_BUILD_UI}" ]]; then if [[ "${DISABLE_BUILD_UI}" == "ON" ]]; then BUILD_UI=0 @@ -492,6 +509,7 @@ echo "Get params: BUILD_BENCHMARK -- ${BUILD_BENCHMARK} BUILD_TASK_EXECUTOR_SIMULATOR -- ${BUILD_TASK_EXECUTOR_SIMULATOR} BUILD_BE_JAVA_EXTENSIONS -- ${BUILD_BE_JAVA_EXTENSIONS} + BUILD_BE_CDC_CLIENT -- ${BUILD_BE_CDC_CLIENT} BUILD_HIVE_UDF -- ${BUILD_HIVE_UDF} PARALLEL -- ${PARALLEL} CLEAN -- ${CLEAN} @@ -975,6 +993,15 @@ if [[ "${BUILD_BROKER}" -eq 1 ]]; then cd "${DORIS_HOME}" fi +if [[ "${BUILD_BE_CDC_CLIENT}" -eq 1 ]]; then + install -d "${DORIS_OUTPUT}/be/lib/cdc_client" + cd "${DORIS_HOME}/fs_brokers/cdc_client" + "${MVN_CMD}" package -DskipTests + rm -rf "${DORIS_OUTPUT}/be/lib/cdc_client"/* + cp -r -p "${DORIS_HOME}/fs_brokers/cdc_client/target/cdc-client.jar" "${DORIS_OUTPUT}/be/lib/cdc_client/" + cd "${DORIS_HOME}" +fi + if [[ ${BUILD_CLOUD} -eq 1 ]]; then rm -rf "${DORIS_HOME}/output/ms" rm -rf "${DORIS_HOME}/cloud/output/lib/hadoop_hdfs" @@ -1000,4 +1027,4 @@ if [[ -n "${DORIS_POST_BUILD_HOOK}" ]]; then eval "${DORIS_POST_BUILD_HOOK}" fi -exit 0 +exit 0 \ No newline at end of file From 8f8f676c155a608fa4b9ccf6344e001b6c350c3f Mon Sep 17 00:00:00 2001 From: wudi Date: Sat, 6 Dec 2025 14:56:50 +0800 Subject: [PATCH 12/27] update check style --- .../insert/streaming/StreamingInsertJob.java | 2 + fs_brokers/cdc_client/pom.xml | 4 ++ .../doris/cdcclient/CdcClientApplication.java | 4 +- .../apache/doris/cdcclient/common/Env.java | 10 ++-- .../config/GlobalExceptionHandler.java | 3 +- .../controller/ClientController.java | 6 ++- .../doris/cdcclient/model/JobConfig.java | 1 + .../model/request/CompareOffsetReq.java | 4 +- .../model/request/FetchTableSplitsReq.java | 3 +- .../model/request/JobBaseRecordReq.java | 4 +- .../model/response/RecordWithMeta.java | 1 + .../model/response/WriteMetaResp.java | 1 + .../service/PipelineCoordinator.java | 29 ++++++----- .../cdcclient/sink/BatchBufferHttpEntity.java | 4 +- .../cdcclient/sink/BatchRecordBuffer.java | 1 + .../cdcclient/sink/DorisBatchStreamLoad.java | 11 ++-- .../doris/cdcclient/sink/HttpPutBuilder.java | 8 +-- .../doris/cdcclient/sink/RespContent.java | 3 +- .../deserialize/DebeziumJsonDeserializer.java | 38 +++++++------- .../source/factory/SourceReaderFactory.java | 6 ++- .../cdcclient/source/reader/SourceReader.java | 5 +- .../source/reader/SplitReadResult.java | 4 +- .../cdcclient/source/reader/SplitRecords.java | 3 +- .../reader/mysql/MySqlSourceReader.java | 51 ++++++++++--------- .../source/split/AbstractSourceSplit.java | 1 + .../cdcclient/source/split/BinlogSplit.java | 1 + .../cdcclient/source/split/SnapshotSplit.java | 10 ++-- .../doris/cdcclient/utils/ConfigUtil.java | 11 ++-- 28 files changed, 140 insertions(+), 89 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 3e016b9bc14195..7e492420653425 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -79,6 +79,8 @@ import lombok.Getter; import lombok.Setter; import lombok.extern.log4j.Log4j2; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; import java.io.DataOutput; import java.io.IOException; diff --git a/fs_brokers/cdc_client/pom.xml b/fs_brokers/cdc_client/pom.xml index 7688139bd65b95..88a6882105df28 100644 --- a/fs_brokers/cdc_client/pom.xml +++ b/fs_brokers/cdc_client/pom.xml @@ -199,6 +199,10 @@ + + org.apache.doris,org.apache,javax,java,scala,\# + + diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java index cd07d0cf73d291..ea000a10141f62 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java @@ -17,8 +17,10 @@ package org.apache.doris.cdcclient; -import java.util.Arrays; import org.apache.doris.cdcclient.common.Env; + +import java.util.Arrays; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.boot.SpringApplication; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java index 685e408b864721..26c4e95047abff 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java @@ -17,16 +17,18 @@ package org.apache.doris.cdcclient.common; +import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.cdcclient.source.factory.DataSource; +import org.apache.doris.cdcclient.source.factory.SourceReaderFactory; +import org.apache.doris.cdcclient.source.reader.SourceReader; + import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; + import lombok.Getter; import lombok.Setter; -import org.apache.doris.cdcclient.model.JobConfig; -import org.apache.doris.cdcclient.source.factory.DataSource; -import org.apache.doris.cdcclient.source.factory.SourceReaderFactory; -import org.apache.doris.cdcclient.source.reader.SourceReader; public class Env { private static volatile Env INSTANCE; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java index 145f83969359eb..30b47a69b38754 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java @@ -1,8 +1,9 @@ package org.apache.doris.cdcclient.config; +import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; + import jakarta.servlet.http.HttpServletRequest; import lombok.extern.slf4j.Slf4j; -import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.context.MessageSource; import org.springframework.web.bind.annotation.ControllerAdvice; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index 51a193a7fc77ff..648f5e1b183bc9 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -17,8 +17,6 @@ package org.apache.doris.cdcclient.controller; -import jakarta.servlet.http.HttpServletRequest; -import java.util.List; import org.apache.doris.cdcclient.common.Env; import org.apache.doris.cdcclient.model.JobConfig; import org.apache.doris.cdcclient.model.request.CompareOffsetReq; @@ -28,6 +26,10 @@ import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; import org.apache.doris.cdcclient.service.PipelineCoordinator; import org.apache.doris.cdcclient.source.reader.SourceReader; + +import java.util.List; + +import jakarta.servlet.http.HttpServletRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java index 921f26449605a6..fe29e280be24ee 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java @@ -18,6 +18,7 @@ package org.apache.doris.cdcclient.model; import java.util.Map; + import lombok.AllArgsConstructor; import lombok.Data; import lombok.NoArgsConstructor; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java index d5eb74834b53a3..f5c596142fca61 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java @@ -17,10 +17,12 @@ package org.apache.doris.cdcclient.model.request; +import org.apache.doris.cdcclient.model.JobConfig; + import java.util.Map; + import lombok.Getter; import lombok.Setter; -import org.apache.doris.cdcclient.model.JobConfig; @Getter @Setter diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java index eaa5106f194e32..957abb198e3e07 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java @@ -17,10 +17,11 @@ package org.apache.doris.cdcclient.model.request; +import org.apache.doris.cdcclient.model.JobConfig; + import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.Setter; -import org.apache.doris.cdcclient.model.JobConfig; @Getter @Setter diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java index dced17b41fb9bb..f84bdf8f11ce37 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java @@ -17,11 +17,13 @@ package org.apache.doris.cdcclient.model.request; +import org.apache.doris.cdcclient.model.JobConfig; + import java.util.Map; + import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.Setter; -import org.apache.doris.cdcclient.model.JobConfig; @Getter @Setter diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java index 621138ee34a81e..4c6d0e4ee7fe00 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; + import lombok.Data; @Data diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/WriteMetaResp.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/WriteMetaResp.java index 9305e1064717c0..a15b9078e3038a 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/WriteMetaResp.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/WriteMetaResp.java @@ -18,6 +18,7 @@ package org.apache.doris.cdcclient.model.response; import java.util.Map; + import lombok.Data; @Data diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 68af2d9856a278..7103dc98d0c3e2 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -17,19 +17,6 @@ package org.apache.doris.cdcclient.service; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import io.debezium.data.Envelope; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import org.apache.commons.collections.CollectionUtils; import org.apache.doris.cdcclient.common.Env; import org.apache.doris.cdcclient.exception.StreamLoadException; import org.apache.doris.cdcclient.model.request.WriteRecordReq; @@ -38,9 +25,25 @@ import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; import org.apache.doris.cdcclient.source.reader.SourceReader; import org.apache.doris.cdcclient.source.reader.SplitReadResult; + +import org.apache.commons.collections.CollectionUtils; import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import io.debezium.data.Envelope; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.stereotype.Component; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferHttpEntity.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferHttpEntity.java index aa93b15cd89f92..04c7a058727f6c 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferHttpEntity.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferHttpEntity.java @@ -17,11 +17,13 @@ package org.apache.doris.cdcclient.sink; +import org.apache.http.entity.AbstractHttpEntity; + import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.List; -import org.apache.http.entity.AbstractHttpEntity; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java index 61beb8eb9c43ef..a81e91a07ee418 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java @@ -18,6 +18,7 @@ package org.apache.doris.cdcclient.sink; import java.util.LinkedList; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java index 63023eced4bf3f..e88e771e0fc657 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java @@ -20,16 +20,14 @@ import org.apache.doris.cdcclient.common.Env; import org.apache.doris.cdcclient.exception.StreamLoadException; import org.apache.doris.cdcclient.utils.HttpUtil; -import com.fasterxml.jackson.databind.ObjectMapper; -import lombok.Setter; + import org.apache.commons.lang3.StringUtils; import org.apache.flink.annotation.VisibleForTesting; import org.apache.http.client.methods.CloseableHttpResponse; import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.util.EntityUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -56,6 +54,11 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.Setter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** async stream load. */ public class DorisBatchStreamLoad implements Serializable { private static final long serialVersionUID = 1L; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java index 337b1e197df7ea..b1ae8c29da6efa 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java @@ -17,16 +17,18 @@ package org.apache.doris.cdcclient.sink; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; import org.apache.doris.cdcclient.constants.LoadConstants; + import org.apache.flink.util.Preconditions; import org.apache.http.HttpEntity; import org.apache.http.HttpHeaders; import org.apache.http.client.methods.HttpPut; import org.apache.http.entity.StringEntity; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + /** Builder for HttpPut. */ public class HttpPutBuilder { String url; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/RespContent.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/RespContent.java index 0a3c8272681c93..35327aa4553cef 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/RespContent.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/RespContent.java @@ -17,11 +17,12 @@ package org.apache.doris.cdcclient.sink; +import org.apache.flink.annotation.VisibleForTesting; + import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.annotation.VisibleForTesting; @JsonIgnoreProperties(ignoreUnknown = true) public class RespContent { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java index e5e387df6f4e0a..5926817ecdb816 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java @@ -17,6 +17,26 @@ package org.apache.doris.cdcclient.source.deserialize; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; +import org.apache.flink.cdc.debezium.utils.TemporalConversions; +import org.apache.flink.table.data.TimestampData; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import java.io.IOException; +import java.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import static org.apache.doris.cdcclient.constants.LoadConstants.DELETE_SIGN_KEY; import com.fasterxml.jackson.core.JsonProcessingException; @@ -29,24 +49,6 @@ import io.debezium.time.NanoTimestamp; import io.debezium.time.Timestamp; import io.debezium.time.ZonedTimestamp; -import java.io.IOException; -import java.math.BigDecimal; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; -import org.apache.flink.cdc.debezium.utils.TemporalConversions; -import org.apache.flink.table.data.TimestampData; -import org.apache.kafka.connect.data.Decimal; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.source.SourceRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java index 507e949498de80..71aadab377fc75 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java @@ -17,12 +17,14 @@ package org.apache.doris.cdcclient.source.factory; +import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.cdcclient.source.reader.mysql.MySqlSourceReader; + import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.function.Supplier; -import org.apache.doris.cdcclient.source.reader.SourceReader; -import org.apache.doris.cdcclient.source.reader.mysql.MySqlSourceReader; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index 05cb5f6e33f27a..0b382e3f88f33c 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -17,8 +17,6 @@ package org.apache.doris.cdcclient.source.reader; -import java.util.List; -import java.util.Map; import org.apache.doris.cdcclient.model.JobConfig; import org.apache.doris.cdcclient.model.request.FetchRecordReq; import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; @@ -26,6 +24,9 @@ import org.apache.doris.cdcclient.model.response.RecordWithMeta; import org.apache.doris.cdcclient.source.split.SourceSplit; +import java.util.List; +import java.util.Map; + /** * SourceReader interface * diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java index b3f9d06edb6ffb..cd96a7e0fc50c5 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java @@ -17,9 +17,11 @@ package org.apache.doris.cdcclient.source.reader; +import org.apache.kafka.connect.source.SourceRecord; + import java.util.Iterator; + import lombok.Data; -import org.apache.kafka.connect.source.SourceRecord; /** * The result of reading a split with iterator. diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java index 905aaf00bc8b33..d7c712e463d116 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java @@ -17,10 +17,11 @@ package org.apache.doris.cdcclient.source.reader; -import java.util.Iterator; import org.apache.flink.cdc.connectors.mysql.source.split.SourceRecords; import org.apache.kafka.connect.source.SourceRecord; +import java.util.Iterator; + public class SplitRecords { private final String splitId; private final SourceRecords records; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index d561896a185ce8..984e8a11ec6c9b 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -17,30 +17,6 @@ package org.apache.doris.cdcclient.source.reader.mysql; -import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.github.shyiko.mysql.binlog.BinaryLogClient; -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.connector.mysql.MySqlPartition; -import io.debezium.document.Array; -import io.debezium.relational.Column; -import io.debezium.relational.TableId; -import io.debezium.relational.history.HistoryRecord; -import io.debezium.relational.history.TableChanges; -import java.io.IOException; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; -import org.apache.commons.collections.CollectionUtils; import org.apache.doris.cdcclient.constants.LoadConstants; import org.apache.doris.cdcclient.model.JobConfig; import org.apache.doris.cdcclient.model.request.FetchRecordReq; @@ -56,6 +32,8 @@ import org.apache.doris.cdcclient.source.split.BinlogSplit; import org.apache.doris.cdcclient.source.split.SnapshotSplit; import org.apache.doris.cdcclient.utils.ConfigUtil; + +import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.utils.Preconditions; import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; @@ -80,6 +58,31 @@ import org.apache.flink.cdc.debezium.history.FlinkJsonTableChangeSerializer; import org.apache.flink.table.types.logical.RowType; import org.apache.kafka.connect.source.SourceRecord; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlPartition; +import io.debezium.document.Array; +import io.debezium.relational.Column; +import io.debezium.relational.TableId; +import io.debezium.relational.history.HistoryRecord; +import io.debezium.relational.history.TableChanges; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java index 6141e05bbc301e..a595bc20e76954 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java @@ -18,6 +18,7 @@ package org.apache.doris.cdcclient.source.split; import java.io.Serializable; + import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java index 2b261bc4939b45..1fdf1353d02b8d 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java @@ -19,6 +19,7 @@ import java.util.List; import java.util.Map; + import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.NoArgsConstructor; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java index 1d63f30d6f6e76..5750bb8f61c95b 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java @@ -17,17 +17,19 @@ package org.apache.doris.cdcclient.source.split; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.cdc.common.utils.Preconditions; + import java.util.List; import java.util.Map; import java.util.Optional; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.Setter; -import org.apache.flink.cdc.common.utils.Preconditions; @Getter @Setter diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index 3634f5086a6655..9a286d78c37cbd 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -17,12 +17,10 @@ package org.apache.doris.cdcclient.utils; -import com.mysql.cj.conf.ConnectionUrl; -import java.util.Map; -import java.util.Properties; -import org.apache.commons.lang3.StringUtils; import org.apache.doris.cdcclient.constants.LoadConstants; import org.apache.doris.cdcclient.model.JobConfig; + +import org.apache.commons.lang3.StringUtils; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; @@ -30,6 +28,11 @@ import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetBuilder; import org.apache.flink.cdc.connectors.mysql.table.StartupOptions; +import java.util.Map; +import java.util.Properties; + +import com.mysql.cj.conf.ConnectionUrl; + public class ConfigUtil { public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { From 59a5cf2b33ebbab11696c56c00b12a8d9f15d514 Mon Sep 17 00:00:00 2001 From: JNSimba <676366545@qq.com> Date: Mon, 8 Dec 2025 22:30:08 +0800 Subject: [PATCH 13/27] fix be fork --- be/src/runtime/cdc_client_manager.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/be/src/runtime/cdc_client_manager.cpp b/be/src/runtime/cdc_client_manager.cpp index 6bdf895ac7c6ab..8ec1c0381ccaa3 100644 --- a/be/src/runtime/cdc_client_manager.cpp +++ b/be/src/runtime/cdc_client_manager.cpp @@ -101,8 +101,8 @@ Status start_cdc_client(PRequestCdcClientResult* result) { const std::string cdc_jar_path = std::string(doris_home) + "/lib/cdc_client/cdc-client.jar"; const std::string cdc_jar_port = "--server.port=" + std::to_string(doris::config::cdc_client_port); - const std::string backend_http_port = std::to_string(config::webserver_port); - const std::string java_opts = "-Xmx2048m -Dlog.path=" + std::string(log_dir); + const std::string backend_http_port = "--backend.http.port=" + std::to_string(config::webserver_port); + const std::string java_opts = "-Dlog.path=" + std::string(log_dir); // check cdc jar exists struct stat buffer; @@ -160,7 +160,7 @@ Status start_cdc_client(PRequestCdcClientResult* result) { << std::endl; std::cout << "Cdc client child process ready to start." << std::endl; std::string java_bin = path + "/bin/java"; - // java -jar -Dlog.path=xx cdc-client.jar --server.port=9096 8040 + // java -jar -Dlog.path=xx cdc-client.jar --server.port=9096 --backend.http.port=8040 execlp(java_bin.c_str(), "java", java_opts.c_str(), "-jar", cdc_jar_path.c_str(), cdc_jar_port.c_str(), backend_http_port.c_str(),(char*)NULL); std::cerr << "Cdc client child process error." << std::endl; From 2b783cd778f401514d775b9374d062c0750760af Mon Sep 17 00:00:00 2001 From: wudi Date: Mon, 8 Dec 2025 22:32:01 +0800 Subject: [PATCH 14/27] fix offset bug --- .../apache/doris/httpv2/rest/LoadAction.java | 12 --- .../doris/httpv2/rest/RestBaseController.java | 13 +++ .../doris/httpv2/rest/StreamingJobAction.java | 12 ++- .../insert/streaming/StreamingInsertJob.java | 33 ++++---- .../streaming/StreamingMultiTblTask.java | 61 +++++++++++++- .../doris/job/offset/jdbc/JdbcOffset.java | 24 +++++- .../offset/jdbc/JdbcSourceOffsetProvider.java | 84 ++++++++++++------- .../job/offset/jdbc/split/BinlogSplit.java | 16 ++-- .../doris/job/util/StreamingJobUtils.java | 21 ++--- .../plans/commands/info/ColumnDefinition.java | 4 + .../doris/cdcclient/CdcClientApplication.java | 3 - .../apache/doris/cdcclient/common/Env.java | 8 +- .../cdcclient/config/BackendPortHolder.java | 19 +++++ .../controller/ClientController.java | 4 +- .../service/PipelineCoordinator.java | 28 ++++--- .../cdcclient/sink/DorisBatchStreamLoad.java | 16 ++-- .../doris/cdcclient/sink/HttpPutBuilder.java | 5 ++ .../cdcclient/source/reader/SourceReader.java | 3 +- .../reader/mysql/MySqlSourceReader.java | 53 +++++++----- .../doris/cdcclient/utils/ConfigUtil.java | 11 +-- .../src/main/resources/application.properties | 3 +- 21 files changed, 293 insertions(+), 140 deletions(-) create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java index 890a41c5710ee1..6a8ca28135f5b6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java @@ -591,18 +591,6 @@ private Pair splitHostAndPort(String hostPort) throws AnalysisE return Pair.of(pair[0], port); } - // NOTE: This function can only be used for AuditlogPlugin stream load for now. - // AuditlogPlugin should be re-disigned carefully, and blow method focuses on - // temporarily addressing the users' needs for audit logs. - // So this function is not widely tested under general scenario - protected boolean checkClusterToken(String token) { - try { - return Env.getCurrentEnv().getTokenManager().checkAuthToken(token); - } catch (UserException e) { - throw new UnauthorizedException(e.getMessage()); - } - } - // NOTE: This function can only be used for AuditlogPlugin stream load for now. // AuditlogPlugin should be re-disigned carefully, and blow method focuses on // temporarily addressing the users' needs for audit logs. diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RestBaseController.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RestBaseController.java index 25ab947c19fc2e..6a87d9d8c975b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RestBaseController.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RestBaseController.java @@ -22,6 +22,7 @@ import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.Config; import org.apache.doris.common.FeConstants; +import org.apache.doris.common.UserException; import org.apache.doris.common.util.NetUtils; import org.apache.doris.httpv2.controller.BaseController; import org.apache.doris.httpv2.entity.ResponseEntityBuilder; @@ -232,6 +233,18 @@ public boolean checkForwardToMaster(HttpServletRequest request) { return !Env.getCurrentEnv().isMaster(); } + // NOTE: This function can only be used for AuditlogPlugin stream load for now. + // AuditlogPlugin should be re-disigned carefully, and blow method focuses on + // temporarily addressing the users' needs for audit logs. + // So this function is not widely tested under general scenario + protected boolean checkClusterToken(String token) { + try { + return Env.getCurrentEnv().getTokenManager().checkAuthToken(token); + } catch (UserException e) { + throw new UnauthorizedException(e.getMessage()); + } + } + private String getRequestBody(HttpServletRequest request) throws IOException { BufferedReader reader = request.getReader(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StreamingJobAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StreamingJobAction.java index ff96bf9365c753..53610142f12bc2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StreamingJobAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StreamingJobAction.java @@ -26,18 +26,21 @@ import com.google.common.base.Strings; import jakarta.servlet.http.HttpServletRequest; import lombok.Getter; +import lombok.NoArgsConstructor; import lombok.Setter; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.springframework.web.bind.annotation.RequestBody; import org.springframework.web.bind.annotation.RequestMapping; import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.RestController; -public class StreamingJobAction extends LoadAction { +@RestController +public class StreamingJobAction extends RestBaseController { private static final Logger LOG = LogManager.getLogger(StreamingJobAction.class); @RequestMapping(path = "/api/streaming/commit_offset", method = RequestMethod.PUT) - public Object streamLoad(HttpServletRequest request, @RequestBody CommitOffsetRequest offsetRequest) { + public Object commitOffset(@RequestBody CommitOffsetRequest offsetRequest, HttpServletRequest request) { String authToken = request.getHeader("token"); // if auth token is not null, check it first if (!Strings.isNullOrEmpty(authToken)) { @@ -64,7 +67,7 @@ private Object updateOffset(CommitOffsetRequest offsetRequest) { StreamingInsertJob streamingJob = (StreamingInsertJob) job; try { - streamingJob.commitOffset(offsetRequest.getTaskId(), offsetRequest.getOffset()); + streamingJob.commitOffset(offsetRequest); return ResponseEntityBuilder.ok("Offset committed successfully"); } catch (Exception e) { LOG.warn("Failed to commit offset for job {}, offset {}: {}", offsetRequest.getJobId(), @@ -75,9 +78,12 @@ private Object updateOffset(CommitOffsetRequest offsetRequest) { @Getter @Setter + @NoArgsConstructor public static class CommitOffsetRequest { public long jobId; public long taskId; public String offset; + public long scannedRows; + public long scannedBytes; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 7e492420653425..32bf44231551a8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -32,6 +32,7 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.httpv2.rest.StreamingJobAction.CommitOffsetRequest; import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.base.TimerDefinition; @@ -73,7 +74,6 @@ import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnStateChangeCallback; - import com.google.common.base.Preconditions; import com.google.gson.annotations.SerializedName; import lombok.Getter; @@ -81,7 +81,6 @@ import lombok.extern.log4j.Log4j2; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; - import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; @@ -200,11 +199,8 @@ private void initSourceJob() { init(); checkRequiredSourceProperties(); List createTbls = createTableIfNotExists(); - this.offsetProvider = new JdbcSourceOffsetProvider(); + this.offsetProvider = new JdbcSourceOffsetProvider(getJobId(), dataSourceType, sourceProperties); JdbcSourceOffsetProvider rdsOffsetProvider = (JdbcSourceOffsetProvider) this.offsetProvider; - rdsOffsetProvider.setJobId(getJobId()); - rdsOffsetProvider.setSourceType(dataSourceType); - rdsOffsetProvider.setSourceProperties(sourceProperties); rdsOffsetProvider.splitChunks(createTbls); } catch (Exception ex) { log.warn("init streaming job for {} failed", dataSourceType, ex); @@ -577,6 +573,15 @@ private void updateCloudJobStatisticAndOffset(StreamingTaskTxnCommitAttachment a offsetProvider.updateOffset(offsetProvider.deserializeOffset(attachment.getOffset())); } + private void updateJobStatisticAndOffset(CommitOffsetRequest offsetRequest) { + if (this.jobStatistic == null) { + this.jobStatistic = new StreamingJobStatistic(); + } + this.jobStatistic.setScannedRows(this.jobStatistic.getScannedRows() + offsetRequest.getScannedRows()); + this.jobStatistic.setLoadBytes(this.jobStatistic.getLoadBytes() + offsetRequest.getScannedBytes()); + offsetProvider.updateOffset(offsetProvider.deserializeOffset(offsetRequest.getOffset())); + } + @Override public void onRegister() throws JobException { Env.getCurrentGlobalTransactionMgr().getCallbackFactory().addCallback(this); @@ -964,7 +969,7 @@ public void gsonPostProcess() throws IOException { if (tvfType != null) { offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(tvfType); } else { - offsetProvider = new JdbcSourceOffsetProvider(); + offsetProvider = new JdbcSourceOffsetProvider(getJobId(), dataSourceType, sourceProperties); } } @@ -1002,7 +1007,8 @@ public void processTimeoutTasks() { writeLock(); try { StreamingMultiTblTask runningMultiTask = (StreamingMultiTblTask) this.runningStreamTask; - if (runningMultiTask.isTimeout()) { + if (TaskStatus.RUNNING.equals(runningMultiTask.getStatus()) + && runningMultiTask.isTimeout()) { runningMultiTask.cancel(false); runningMultiTask.setErrMsg("task cancelled cause timeout"); @@ -1019,23 +1025,22 @@ public void processTimeoutTasks() { } } - public void commitOffset(long taskId, String offsetStr) throws JobException { + public void commitOffset(CommitOffsetRequest offsetRequest) throws JobException { if (!(offsetProvider instanceof JdbcSourceOffsetProvider)) { throw new JobException("Unsupported commit offset for offset provider type: " + offsetProvider.getClass().getSimpleName()); } writeLock(); try { - Offset offset = offsetProvider.deserializeOffset(offsetStr); - offsetProvider.updateOffset(offset); + updateJobStatisticAndOffset(offsetRequest); if (this.runningStreamTask != null && this.runningStreamTask instanceof StreamingMultiTblTask) { - if (this.runningStreamTask.getTaskId() != taskId) { + if (this.runningStreamTask.getTaskId() != offsetRequest.getTaskId()) { throw new JobException("Task id mismatch when commit offset. expected: " - + this.runningStreamTask.getTaskId() + ", actual: " + taskId); + + this.runningStreamTask.getTaskId() + ", actual: " + offsetRequest.getTaskId()); } persistOffsetProviderIfNeed(); - onStreamTaskSuccess(this.runningStreamTask); + ((StreamingMultiTblTask) this.runningStreamTask).successCallback(offsetRequest); } } finally { writeUnlock(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java index f6fe12386dce34..c47aa4b4e31d38 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java @@ -20,25 +20,29 @@ import org.apache.doris.catalog.Env; import org.apache.doris.httpv2.entity.ResponseBody; import org.apache.doris.httpv2.rest.RestApiStatusCode; +import org.apache.doris.httpv2.rest.StreamingJobAction.CommitOffsetRequest; +import org.apache.doris.job.base.Job; import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.offset.SourceOffsetProvider; import org.apache.doris.job.offset.jdbc.JdbcOffset; +import org.apache.doris.job.offset.jdbc.JdbcSourceOffsetProvider; +import org.apache.doris.job.offset.jdbc.split.BinlogSplit; +import org.apache.doris.job.offset.jdbc.split.SnapshotSplit; import org.apache.doris.job.util.StreamingJobUtils; -import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.proto.InternalService; import org.apache.doris.proto.InternalService.PRequestCdcClientResult; import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.system.Backend; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TStatusCode; - +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; import lombok.Getter; import lombok.extern.log4j.Log4j2; - import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -55,6 +59,8 @@ public class StreamingMultiTblTask extends AbstractStreamingTask { private Map targetProperties; private String targetDb; private StreamingJobProperties jobProperties; + private long scannedRows = 0L; + private long scannedBytes = 0L; public StreamingMultiTblTask(Long jobId, long taskId, @@ -70,6 +76,7 @@ public StreamingMultiTblTask(Long jobId, this.sourceProperties = sourceProperties; this.targetProperties = targetProperties; this.jobProperties = jobProperties; + this.targetDb = targetDb; } @Override @@ -99,7 +106,7 @@ private void sendWriteRequest() throws JobException { Map params = buildRequestParams(); InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() .setApi("/api/writeRecords") - .setParams(GsonUtils.GSON.toJson(params)).build(); + .setParams(new Gson().toJson(params)).build(); TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); InternalService.PRequestCdcClientResult result = null; try { @@ -168,6 +175,52 @@ public boolean onSuccess() throws JobException { return false; } + /** + * Callback function for offset commit success. + */ + public void successCallback(CommitOffsetRequest offsetRequest) { + if (getIsCanceled().get()) { + return; + } + this.status = TaskStatus.SUCCESS; + this.finishTimeMs = System.currentTimeMillis(); + JdbcOffset runOffset = (JdbcOffset) this.runningOffset; + + // set end offset to running offset + Map offsetMeta; + try { + offsetMeta = objectMapper.readValue(offsetRequest.getOffset(), new TypeReference>() { + }); + } catch (JsonProcessingException e) { + log.warn("Failed to parse offset meta from request: {}", offsetRequest.getOffset(), e); + throw new RuntimeException(e); + } + String splitId = offsetMeta.remove(JdbcSourceOffsetProvider.SPLIT_ID); + if (runOffset.getSplit().snapshotSplit() + && !BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)) { + SnapshotSplit split = (SnapshotSplit) runOffset.getSplit(); + split.setHighWatermark(offsetMeta); + } else if (!runOffset.getSplit().snapshotSplit() + && BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)) { + BinlogSplit split = (BinlogSplit) runOffset.getSplit(); + split.setEndingOffset(offsetMeta); + } else { + log.warn("Split id is not consistent, task running split id {}," + + " offset commit request split id {}", runOffset.getSplit().getSplitId(), splitId); + throw new RuntimeException("Split id is not consistent"); + } + if (!isCallable()) { + return; + } + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + if (null == job) { + log.info("job is null, job id is {}", jobId); + return; + } + StreamingInsertJob streamingInsertJob = (StreamingInsertJob) job; + streamingInsertJob.onStreamTaskSuccess(this); + } + @Override protected void onFail(String errMsg) throws JobException { super.onFail(errMsg); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java index da9c84bc909aa7..a71faf4e60d019 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -19,11 +19,15 @@ import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.jdbc.split.AbstractSourceSplit; - +import org.apache.doris.job.offset.jdbc.split.BinlogSplit; +import com.google.gson.Gson; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.Setter; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.collections4.MapUtils; +import java.util.HashMap; @Getter @Setter @@ -50,6 +54,22 @@ public boolean isValidOffset() { @Override public String showRange() { - return null; + if (split.snapshotSplit()) { + // need to show hw + return new Gson().toJson(split); + }else { + BinlogSplit binlogSplit = (BinlogSplit) split; + HashMap showMap = new HashMap<>(); + showMap.put(JdbcSourceOffsetProvider.SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); + if (binlogSplit.getStartingOffset() != null) { + showMap.put("startOffset", binlogSplit.getStartingOffset()); + } else if (CollectionUtils.isNotEmpty(binlogSplit.getFinishedSplits())) { + showMap.put("finishedSplitSize", binlogSplit.getFinishedSplits().size()); + } + if (MapUtils.isNotEmpty(binlogSplit.getEndingOffset())) { + showMap.put("endOffset", binlogSplit.getEndingOffset()); + } + return new Gson().toJson(showMap); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index f9ceb879c49436..2a3cac19c21985 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -37,7 +37,6 @@ import org.apache.doris.system.Backend; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TStatusCode; - import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -48,7 +47,6 @@ import lombok.extern.log4j.Log4j2; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.MapUtils; - import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -63,15 +61,14 @@ @Setter @Log4j2 public class JdbcSourceOffsetProvider implements SourceOffsetProvider { - public static final String BINLOG_SPLIT_ID = "binlog-split"; public static final String SPLIT_ID = "splitId"; private static final ObjectMapper objectMapper = new ObjectMapper(); private Long jobId; private DataSourceType sourceType; private Map sourceProperties = new HashMap<>(); - List remainingSplits; - List finishedSplits; + List remainingSplits = new ArrayList<>(); + List finishedSplits = new ArrayList<>(); JdbcOffset currentOffset; Map endBinlogOffset; @@ -79,9 +76,16 @@ public class JdbcSourceOffsetProvider implements SourceOffsetProvider { @SerializedName("chw") // tableID -> splitId -> chunk of highWatermark Map>> chunkHighWatermarkMap; - @SerializedName("chw") + @SerializedName("bop") Map binlogOffsetPersist; + public JdbcSourceOffsetProvider(Long jobId, DataSourceType sourceType, Map sourceProperties) { + this.jobId = jobId; + this.sourceType = sourceType; + this.sourceProperties = sourceProperties; + this.chunkHighWatermarkMap = new HashMap<>(); + } + @Override public String getSourceType() { return "jdbc"; @@ -95,16 +99,15 @@ public Offset getNextOffset(StreamingJobProperties jobProps, Map SnapshotSplit snapshotSplit = remainingSplits.get(0); nextOffset.setSplit(snapshotSplit); return nextOffset; - } else if (currentOffset.getSplit().snapshotSplit()) { + } else if (currentOffset != null && currentOffset.getSplit().snapshotSplit()) { // snapshot to binlog BinlogSplit binlogSplit = new BinlogSplit(); - binlogSplit.setSplitId(BINLOG_SPLIT_ID); binlogSplit.setFinishedSplits(finishedSplits); nextOffset.setSplit(binlogSplit); return nextOffset; } else { // only binlog - return currentOffset; + return currentOffset == null ? new JdbcOffset(new BinlogSplit()) : currentOffset; } } @@ -113,10 +116,22 @@ public String getShowCurrentOffset() { if (this.currentOffset != null) { AbstractSourceSplit split = this.currentOffset.getSplit(); if (split.snapshotSplit()) { - return new Gson().toJson(split); + SnapshotSplit snsplit = (SnapshotSplit) split; + Map splitShow = new HashMap<>(); + splitShow.put("splitId", snsplit.getSplitId()); + splitShow.put("tableId", snsplit.getTableId()); + splitShow.put("splitKey", snsplit.getSplitKey()); + splitShow.put("splitStart", snsplit.getSplitStart()); + splitShow.put("splitEnd", snsplit.getSplitEnd()); + return new Gson().toJson(splitShow); } else { BinlogSplit binlogSplit = (BinlogSplit) split; - return new Gson().toJson(binlogSplit.getStartingOffset()); + HashMap showMap = new HashMap<>(); + showMap.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); + if (binlogSplit.getStartingOffset() != null) { + showMap.putAll(binlogSplit.getStartingOffset()); + } + return new Gson().toJson(showMap); } } return null; @@ -162,7 +177,7 @@ public void updateOffset(Offset offset) { } else { BinlogSplit binlogSplit = (BinlogSplit) split; binlogOffsetPersist = new HashMap<>(binlogSplit.getStartingOffset()); - binlogOffsetPersist.put(SPLIT_ID, BINLOG_SPLIT_ID); + binlogOffsetPersist.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); } } @@ -172,7 +187,7 @@ public void fetchRemoteMeta(Map properties) throws Exception { Map params = buildBaseParams(); InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() .setApi("/api/fetchEndOffset") - .setParams(GsonUtils.GSON.toJson(params)).build(); + .setParams(new Gson().toJson(params)).build(); TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); InternalService.PRequestCdcClientResult result = null; try { @@ -201,20 +216,31 @@ public void fetchRemoteMeta(Map properties) throws Exception { @Override public boolean hasMoreDataToConsume() { - if (!remainingSplits.isEmpty()) { + if (currentOffset == null) { return true; } - if (currentOffset != null && !currentOffset.getSplit().snapshotSplit()) { - BinlogSplit binlogSplit = (BinlogSplit) currentOffset.getSplit(); - Map startingOffset = binlogSplit.getStartingOffset(); - try { - return compareOffset(startingOffset, endBinlogOffset); - } catch (Exception ex) { - log.info("Compare offset error: ", ex); - return false; + + if (CollectionUtils.isNotEmpty(remainingSplits)) { + return true; + } + if (MapUtils.isEmpty(endBinlogOffset)) { + return false; + } + try { + if (!currentOffset.getSplit().snapshotSplit()) { + BinlogSplit binlogSplit = (BinlogSplit) currentOffset.getSplit(); + return compareOffset(endBinlogOffset, new HashMap<>(binlogSplit.getStartingOffset())); + } else { + SnapshotSplit snapshotSplit = (SnapshotSplit) currentOffset.getSplit(); + if (MapUtils.isNotEmpty(snapshotSplit.getHighWatermark())) { + return compareOffset(endBinlogOffset, new HashMap<>(snapshotSplit.getHighWatermark())); + } } + } catch (Exception ex) { + log.info("Compare offset error: ", ex); + return false; } - return true; + return false; } private boolean compareOffset(Map offsetFirst, Map offsetSecond) @@ -223,7 +249,7 @@ private boolean compareOffset(Map offsetFirst, Map params = buildCompareOffsetParams(offsetFirst, offsetSecond); InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() .setApi("/api/compareOffset") - .setParams(GsonUtils.GSON.toJson(params)).build(); + .setParams(new Gson().toJson(params)).build(); TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); InternalService.PRequestCdcClientResult result = null; try { @@ -257,7 +283,7 @@ public Offset deserializeOffset(String offset) { Map offsetMeta = objectMapper.readValue(offset, new TypeReference>() { }); String splitId = offsetMeta.remove(SPLIT_ID); - if (BINLOG_SPLIT_ID.equals(splitId)) { + if (BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)) { BinlogSplit binlogSplit = new BinlogSplit(); binlogSplit.setSplitId(splitId); binlogSplit.setStartingOffset(offsetMeta); @@ -285,13 +311,13 @@ public Offset deserializeOffsetProperty(String offset) { */ @Override public void replayIfNeed(StreamingInsertJob job) { - if (remainingSplits == null - && checkNeedSplitChunks(job.getSourceProperties())) { + if (checkNeedSplitChunks(job.getSourceProperties())) { try { Map> snapshotSplits = StreamingJobUtils.restoreSplitsToJob(job.getJobId()); String offsetProviderPersist = job.getOffsetProviderPersist(); JdbcSourceOffsetProvider replayFromPersist = GsonUtils.GSON.fromJson(offsetProviderPersist, JdbcSourceOffsetProvider.class); + // need a flag to check pure binlog? this.chunkHighWatermarkMap = replayFromPersist.getChunkHighWatermarkMap(); if (MapUtils.isNotEmpty(chunkHighWatermarkMap) && MapUtils.isNotEmpty(snapshotSplits)) { replaySnapshotSplits(chunkHighWatermarkMap, snapshotSplits); @@ -299,7 +325,7 @@ && checkNeedSplitChunks(job.getSourceProperties())) { this.binlogOffsetPersist = replayFromPersist.getBinlogOffsetPersist(); if (MapUtils.isNotEmpty(binlogOffsetPersist)) { currentOffset = new JdbcOffset(); - currentOffset.setSplit(new BinlogSplit(BINLOG_SPLIT_ID, binlogOffsetPersist)); + currentOffset.setSplit(new BinlogSplit(binlogOffsetPersist)); } } catch (Exception ex) { log.error("Failed to restore splits for job {}", job.getJobId(), ex); @@ -383,7 +409,7 @@ private List requestTableSplits(String table) throws JobException Map params = buildSplitParams(table); InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() .setApi("/api/fetchSplits") - .setParams(GsonUtils.GSON.toJson(params)).build(); + .setParams(new Gson().toJson(params)).build(); TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); InternalService.PRequestCdcClientResult result = null; try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java index bd082be2082733..a1bb22d54a0965 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java @@ -20,28 +20,27 @@ import com.google.gson.Gson; import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.NoArgsConstructor; import lombok.Setter; - import java.util.List; import java.util.Map; @Getter @Setter @EqualsAndHashCode(callSuper = true) -@NoArgsConstructor public class BinlogSplit extends AbstractSourceSplit { private static final long serialVersionUID = 1L; - + public static final String BINLOG_SPLIT_ID = "binlog-split"; private Map startingOffset; private Map endingOffset; // binlog split meta, first binlog split requires private List finishedSplits; - public BinlogSplit( - String splitId, - Map startingOffset) { - this.splitId = splitId; + public BinlogSplit() { + this.splitId = BINLOG_SPLIT_ID; + } + + public BinlogSplit(Map startingOffset) { + this.splitId = BINLOG_SPLIT_ID; this.startingOffset = startingOffset; } @@ -49,4 +48,5 @@ public BinlogSplit( public String toString() { return new Gson().toJson(this); } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java index 2785c7b9b00e50..695008ba51bd14 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java @@ -18,7 +18,6 @@ package org.apache.doris.job.util; import org.apache.doris.analysis.UserIdentity; -import org.apache.doris.catalog.AggregateType; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; @@ -47,14 +46,13 @@ import org.apache.doris.system.BeSelectionPolicy; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TUniqueId; - import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import lombok.extern.log4j.Log4j2; import org.apache.commons.text.StringSubstitutor; - import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -264,12 +262,7 @@ public static List generateCreateTableInfos(String targetDb, Da // Convert Column to ColumnDefinition List columnDefinitions = columns.stream().map(col -> { DataType dataType = DataType.fromCatalogType(col.getType()); - boolean isKey = col.isKey(); - AggregateType aggType = col.getAggregationType(); - boolean isNullable = col.isAllowNull(); - String comment = col.getComment() != null ? col.getComment() : ""; - return new ColumnDefinition(col.getName(), dataType, isKey, aggType, isNullable, - Optional.empty(), comment); + return new ColumnDefinition(col.getName(), dataType, col.isKey(), col.isAllowNull(), col.getComment()); }).collect(Collectors.toList()); // Create DistributionDescriptor @@ -289,17 +282,17 @@ public static List generateCreateTableInfos(String targetDb, Da targetDb, // dbName table, // tableName columnDefinitions, // columns - Collections.emptyList(), // indexes + ImmutableList.of(), // indexes "olap", // engineName KeysType.UNIQUE_KEYS, // keysType primaryKeys, // keys "", // comment PartitionTableInfo.EMPTY, // partitionTableInfo distribution, // distribution - Collections.emptyList(), // rollups + ImmutableList.of(), // rollups tableCreateProperties, // properties - new HashMap<>(), // extProperties - Collections.emptyList() // clusterKeyColumnNames + ImmutableMap.of(), // extProperties + ImmutableList.of() // clusterKeyColumnNames ); createtblInfo.analyzeEngine(); createtblInfos.add(createtblInfo); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java index d88a3c4a5d08e1..98cdb9014a5539 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java @@ -144,6 +144,10 @@ public ColumnDefinition(String name, DataType type, boolean isNullable, String c this(name, type, false, null, isNullable, Optional.empty(), comment); } + public ColumnDefinition(String name, DataType type, boolean isKey, boolean isNullable, String comment) { + this(name, type, isKey, null, isNullable, Optional.empty(), comment); + } + public String getName() { return name; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java index ea000a10141f62..0547dea6741246 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java @@ -17,8 +17,6 @@ package org.apache.doris.cdcclient; -import org.apache.doris.cdcclient.common.Env; - import java.util.Arrays; import org.slf4j.Logger; @@ -37,7 +35,6 @@ public class CdcClientApplication extends SpringBootServletInitializer { public static void main(String[] args) { LOG.info("args: {}, log.path: {}", Arrays.asList(args), System.getProperty("log.path")); - Env.getCurrentEnv().setBackendHostPort("10.16.10.6:28747"); SpringApplication.run(CdcClientApplication.class, args); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java index 26c4e95047abff..aabdaf1df4d1b4 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java @@ -27,18 +27,22 @@ import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; -import lombok.Getter; import lombok.Setter; public class Env { private static volatile Env INSTANCE; private final Map jobContexts; - @Getter @Setter private String backendHostPort; + @Setter private int backendHttpPort; private Env() { this.jobContexts = new ConcurrentHashMap<>(); } + public String getBackendHostPort() { + return "127.0.0.1:" + backendHttpPort; + // return "10.16.10.6:28949"; + } + public static Env getCurrentEnv() { if (INSTANCE == null) { synchronized (Env.class) { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java new file mode 100644 index 00000000000000..9541479e44fc6f --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java @@ -0,0 +1,19 @@ +package org.apache.doris.cdcclient.config; + +import org.apache.doris.cdcclient.common.Env; + +import jakarta.annotation.PostConstruct; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.stereotype.Component; + +@Component +public class BackendPortHolder { + + @Value("${backend.http.port}") + private int port; + + @PostConstruct + public void init() { + Env.getCurrentEnv().setBackendHttpPort(port); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index 648f5e1b183bc9..fe6f6ffce5afe6 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -92,9 +92,7 @@ public Object fetchEndOffset(@RequestBody JobConfig jobConfig) { @RequestMapping(path = "/api/compareOffset", method = RequestMethod.POST) public Object compareOffset(@RequestBody CompareOffsetReq compareOffsetReq) { SourceReader reader = Env.getCurrentEnv().getReader(compareOffsetReq); - return ResponseEntityBuilder.ok( - reader.compareOffset( - compareOffsetReq.getOffsetFirst(), compareOffsetReq.getOffsetSecond())); + return ResponseEntityBuilder.ok(reader.compareOffset(compareOffsetReq)); } @RequestMapping(path = "/api/close/{jobId}", method = RequestMethod.POST) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 7103dc98d0c3e2..acc4a5b31c5b31 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -25,12 +25,16 @@ import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; import org.apache.doris.cdcclient.source.reader.SourceReader; import org.apache.doris.cdcclient.source.reader.SplitReadResult; - +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import io.debezium.data.Envelope; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; - +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Component; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -41,13 +45,6 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import io.debezium.data.Envelope; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.stereotype.Component; - /** Pipeline coordinator. */ @Component public class PipelineCoordinator { @@ -129,6 +126,8 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { long startTime = System.currentTimeMillis(); long maxIntervalMillis = writeRecordReq.getMaxInterval() * 1000; + long scannedRows = 0L; + long scannedBytes = 0L; // Use iterators to read and write. Iterator iterator = readResult.getRecordIterator(); while (iterator != null && iterator.hasNext()) { @@ -141,17 +140,22 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { String table = extractTable(element); hasData = true; for (String record : serializedRecords) { - batchStreamLoad.writeRecord(database, table, record.getBytes()); + scannedRows++; + byte[] dataBytes = record.getBytes(); + scannedBytes += dataBytes.length; + batchStreamLoad.writeRecord(database, table, dataBytes); } Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); if (readBinlog && sourceReader.getSplitId(readResult.getSplit()) != null) { lastMeta.put(SPLIT_ID, sourceReader.getSplitId(readResult.getSplit())); - lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + // lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); } metaResponse = lastMeta; } + } else { + LOG.info("Skip non-data record: {}", element.valueSchema()); } // Check if maxInterval has been exceeded long elapsedTime = System.currentTimeMillis() - startTime; @@ -182,7 +186,7 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { metaResponse = offsetRes; } // request fe api - batchStreamLoad.commitOffset(metaResponse); + batchStreamLoad.commitOffset(metaResponse, scannedRows, scannedBytes); // batchStreamLoad.commitTransaction(metaResponse); } finally { sourceReader.finishSplitRecords(); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java index e88e771e0fc657..00e63a43a284d7 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java @@ -356,6 +356,7 @@ public void run() { public void load(String label, BatchRecordBuffer buffer) throws IOException { BatchBufferHttpEntity entity = new BatchBufferHttpEntity(buffer); HttpPutBuilder putBuilder = new HttpPutBuilder(); + String loadUrl = String.format(LOAD_URL_PATTERN, hostPort, targetDb, buffer.getTable()); String finalLabel = String.format("%s_%s_%s", jobId, currentTaskId, label); putBuilder @@ -371,6 +372,7 @@ public void load(String label, BatchRecordBuffer buffer) throws IOException { Throwable resEx = new Throwable(); int retry = 0; while (retry <= RETRY) { + LOG.info("stream load started for {} on host {}", putBuilder.getLabel(), hostPort); try (CloseableHttpClient httpClient = HttpUtil.getHttpClient()) { try (CloseableHttpResponse response = httpClient.execute(putBuilder.build())) { int statusCode = response.getStatusLine().getStatusCode(); @@ -464,24 +466,28 @@ public void resetTaskId() { } /** commit offfset to frontends. */ - public void commitOffset(Map meta) { + public void commitOffset(Map meta, long scannedRows, long scannedBytes) { try { String url = String.format(COMMIT_URL_PATTERN, frontendAddress, targetDb); Map commitParams = new HashMap<>(); commitParams.put("offset", OBJECT_MAPPER.writeValueAsString(meta)); commitParams.put("jobId", jobId); commitParams.put("taskId", currentTaskId); + commitParams.put("scannedRows", scannedRows); + commitParams.put("scannedBytes", scannedBytes); + String param = OBJECT_MAPPER.writeValueAsString(commitParams); + HttpPutBuilder builder = new HttpPutBuilder() .addCommonHeader() + .addBodyContentType() .addTokenAuth(token) .setUrl(url) .commit() - .setEntity( - new StringEntity( - OBJECT_MAPPER.writeValueAsString(commitParams))); + .setEntity(new StringEntity(param)); - LOG.info("commit offset for jobId {} taskId {}", jobId, currentTaskId); + LOG.info( + "commit offset for jobId {} taskId {}, params {}", jobId, currentTaskId, param); Throwable resEx = null; int retry = 0; while (retry <= RETRY) { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java index b1ae8c29da6efa..7aa177f1f2ef74 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java @@ -49,6 +49,11 @@ public HttpPutBuilder addCommonHeader() { return this; } + public HttpPutBuilder addBodyContentType() { + header.put(HttpHeaders.CONTENT_TYPE, "application/json;charset=UTF-8"); + return this; + } + public HttpPutBuilder addHiddenColumns(boolean add) { if (add) { header.put("hidden_columns", LoadConstants.DELETE_SIGN_KEY); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index 0b382e3f88f33c..cb586823d5d928 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -18,6 +18,7 @@ package org.apache.doris.cdcclient.source.reader; import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.cdcclient.model.request.CompareOffsetReq; import org.apache.doris.cdcclient.model.request.FetchRecordReq; import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; import org.apache.doris.cdcclient.model.request.JobBaseRecordReq; @@ -72,5 +73,5 @@ default SplitReadResult readSplitRecords(JobBaseRecordReq bas Map getEndOffset(JobConfig jobConfig); - int compareOffset(Map offsetFirst, Map offsetSecond); + int compareOffset(CompareOffsetReq compareOffsetReq); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index 984e8a11ec6c9b..be96f90cbdd929 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -19,6 +19,7 @@ import org.apache.doris.cdcclient.constants.LoadConstants; import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.cdcclient.model.request.CompareOffsetReq; import org.apache.doris.cdcclient.model.request.FetchRecordReq; import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; import org.apache.doris.cdcclient.model.request.JobBaseRecordReq; @@ -32,7 +33,16 @@ import org.apache.doris.cdcclient.source.split.BinlogSplit; import org.apache.doris.cdcclient.source.split.SnapshotSplit; import org.apache.doris.cdcclient.utils.ConfigUtil; - +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlPartition; +import io.debezium.document.Array; +import io.debezium.relational.Column; +import io.debezium.relational.TableId; +import io.debezium.relational.history.HistoryRecord; +import io.debezium.relational.history.TableChanges; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.utils.Preconditions; @@ -41,6 +51,7 @@ import org.apache.flink.cdc.connectors.mysql.debezium.reader.DebeziumReader; import org.apache.flink.cdc.connectors.mysql.debezium.reader.SnapshotSplitReader; import org.apache.flink.cdc.connectors.mysql.debezium.task.context.StatefulTaskContext; +import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; import org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlSnapshotSplitAssigner; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; @@ -58,7 +69,8 @@ import org.apache.flink.cdc.debezium.history.FlinkJsonTableChangeSerializer; import org.apache.flink.table.types.logical.RowType; import org.apache.kafka.connect.source.SourceRecord; - +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.sql.SQLException; import java.util.ArrayList; @@ -71,21 +83,6 @@ import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; -import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.github.shyiko.mysql.binlog.BinaryLogClient; -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.connector.mysql.MySqlPartition; -import io.debezium.document.Array; -import io.debezium.relational.Column; -import io.debezium.relational.TableId; -import io.debezium.relational.history.HistoryRecord; -import io.debezium.relational.history.TableChanges; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - public class MySqlSourceReader implements SourceReader { private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceReader.class); private static ObjectMapper objectMapper = new ObjectMapper(); @@ -243,7 +240,7 @@ private RecordWithMeta buildRecordResponse( Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); if (readBinlog) { lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); - lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + // lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); recordResponse.setMeta(lastMeta); } if (count >= fetchRecord.getFetchSize()) { @@ -271,7 +268,7 @@ private RecordWithMeta buildRecordResponse( offsetRes = split.asBinlogSplit().getStartingOffset().getOffset(); } offsetRes.put(SPLIT_ID, BINLOG_SPLIT_ID); - offsetRes.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + // offsetRes.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); recordResponse.setMeta(offsetRes); } else { SnapshotSplit snapshotSplit = @@ -474,7 +471,7 @@ private SplitRecords pollSplitRecordsWithSplit(MySqlSplit split, JobConfig jobCo currentReader.submitSplit(split); currentSplitId = split.splitId(); // make split record available - Thread.sleep(100); + Thread.sleep(1000); dataIt = currentReader.pollSplitRecords(); if (currentReader instanceof SnapshotSplitReader) { closeSnapshotReader(); @@ -581,7 +578,7 @@ public Map extractBinlogOffset(Object split, boolean pureBinlogP MySqlSplit mysqlSplit = (MySqlSplit) split; Map offsetRes = mysqlSplit.asBinlogSplit().getStartingOffset().getOffset(); offsetRes.put(SPLIT_ID, BINLOG_SPLIT_ID); - offsetRes.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + // offsetRes.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); return offsetRes; } @@ -617,7 +614,19 @@ public Map getEndOffset(JobConfig jobConfig) { } @Override - public int compareOffset(Map offsetFirst, Map offsetSecond) { + public int compareOffset(CompareOffsetReq compareOffsetReq) { + Map offsetFirst = compareOffsetReq.getOffsetFirst(); + Map offsetSecond = compareOffsetReq.getOffsetSecond(); + // make server id is equals + String serverId1 = offsetFirst.get("server_id"); + String serverId2 = offsetSecond.get("server_id"); + if (serverId1 == null && serverId2 != null) { + offsetFirst.put("server_id", serverId2); + } + if (serverId2 == null && serverId1 != null) { + offsetSecond.put("server_id", serverId1); + } + BinlogOffset binlogOffset1 = new BinlogOffset(offsetFirst); BinlogOffset binlogOffset2 = new BinlogOffset(offsetSecond); return binlogOffset1.compareTo(binlogOffset2); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index 9a286d78c37cbd..a2ef64dda2dcbb 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -19,7 +19,7 @@ import org.apache.doris.cdcclient.constants.LoadConstants; import org.apache.doris.cdcclient.model.JobConfig; - +import com.mysql.cj.conf.ConnectionUrl; import org.apache.commons.lang3.StringUtils; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory; @@ -27,14 +27,15 @@ import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetBuilder; import org.apache.flink.cdc.connectors.mysql.table.StartupOptions; - import java.util.Map; import java.util.Properties; -import com.mysql.cj.conf.ConnectionUrl; - public class ConfigUtil { + public static String getServerId(long jobId) { + return String.valueOf(Math.abs(String.valueOf(jobId).hashCode())); + } + public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { Map cdcConfig = config.getConfig(); MySqlSourceConfigFactory configFactory = new MySqlSourceConfigFactory(); @@ -47,7 +48,7 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { configFactory.password(cdcConfig.get(LoadConstants.PASSWORD)); String databaseName = cdcConfig.get(LoadConstants.DATABASE); configFactory.databaseList(databaseName); - configFactory.serverId(String.valueOf(Math.abs(config.getJobId().hashCode()))); + configFactory.serverId(getServerId(config.getJobId())); configFactory.includeSchemaChanges(false); diff --git a/fs_brokers/cdc_client/src/main/resources/application.properties b/fs_brokers/cdc_client/src/main/resources/application.properties index 95a974099af040..2dc3cb46ef2f49 100644 --- a/fs_brokers/cdc_client/src/main/resources/application.properties +++ b/fs_brokers/cdc_client/src/main/resources/application.properties @@ -1,2 +1,3 @@ spring.web.resources.add-mappings=false -server.port=9096 \ No newline at end of file +server.port=9096 +backend.http.port=8040 \ No newline at end of file From 79a71edd13a42c98ecb9916d5ab5dce81cc22fb4 Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 9 Dec 2025 21:44:27 +0800 Subject: [PATCH 15/27] fix create table and offset consumer bug --- .../datasource/jdbc/client/JdbcClient.java | 6 +- .../apache/doris/httpv2/rest/LoadAction.java | 1 - .../streaming/AbstractStreamingTask.java | 35 +---- .../insert/streaming/StreamingInsertJob.java | 42 ++++-- .../insert/streaming/StreamingInsertTask.java | 43 +++++- .../streaming/StreamingJobSchedulerTask.java | 2 +- .../streaming/StreamingMultiTblTask.java | 40 ++++- .../job/offset/SourceOffsetProvider.java | 3 +- .../doris/job/offset/jdbc/JdbcOffset.java | 4 +- .../offset/jdbc/JdbcSourceOffsetProvider.java | 71 ++++++--- .../job/offset/jdbc/split/BinlogSplit.java | 1 + .../doris/job/util/StreamingJobUtils.java | 59 ++++---- .../plans/commands/info/ColumnDefinition.java | 4 - .../cdcclient/config/BackendPortHolder.java | 18 ++- .../config/GlobalExceptionHandler.java | 17 +++ .../model/request/FetchRecordReq.java | 2 +- .../service/PipelineCoordinator.java | 34 +++-- .../deserialize/DebeziumJsonDeserializer.java | 2 +- .../cdcclient/source/reader/SourceReader.java | 2 +- .../reader/mysql/MySqlSourceReader.java | 141 +++++++----------- .../cdcclient/source/split/BinlogSplit.java | 12 +- .../cdcclient/source/split/SnapshotSplit.java | 49 ------ .../doris/cdcclient/utils/ConfigUtil.java | 39 ++++- 23 files changed, 353 insertions(+), 274 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java index 53d2a7116bd947..0669b7c91099b5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java @@ -394,19 +394,17 @@ public List getJdbcColumnsInfo(String remoteDbName, String remo public List getColumnsFromJdbc(String remoteDbName, String remoteTableName) { List jdbcTableSchema = getJdbcColumnsInfo(remoteDbName, remoteTableName); - List primaryKeys = getPrimaryKeys(remoteDbName, remoteTableName); List dorisTableSchema = Lists.newArrayListWithCapacity(jdbcTableSchema.size()); for (JdbcFieldSchema field : jdbcTableSchema) { - boolean isKey = primaryKeys.contains(field.getColumnName()); dorisTableSchema.add(new Column(field.getColumnName(), - jdbcTypeToDoris(field), isKey, null, + jdbcTypeToDoris(field), true, null, field.isAllowNull(), field.getRemarks(), true, -1)); } return dorisTableSchema; } - private List getPrimaryKeys(String remoteDbName, String remoteTableName) { + public List getPrimaryKeys(String remoteDbName, String remoteTableName) { Connection conn = getConnection(); ResultSet rs = null; List primaryKeys = Lists.newArrayList(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java index 6a8ca28135f5b6..571a6dc50dee9d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java @@ -29,7 +29,6 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.LoadException; import org.apache.doris.common.Pair; -import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugPointUtil; import org.apache.doris.httpv2.entity.ResponseEntityBuilder; import org.apache.doris.httpv2.entity.RestBaseResult; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java index 6554c6d6b23103..2618d8a122cc19 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java @@ -25,7 +25,6 @@ import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.offset.Offset; -import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; @@ -34,8 +33,6 @@ import lombok.extern.log4j.Log4j2; import org.apache.commons.lang3.StringUtils; -import java.util.Arrays; -import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; @Log4j2 @@ -59,9 +56,10 @@ public abstract class AbstractStreamingTask { @Getter private AtomicBoolean isCanceled = new AtomicBoolean(false); - public AbstractStreamingTask(long jobId, long taskId) { + public AbstractStreamingTask(long jobId, long taskId, UserIdentity userIdentity) { this.jobId = jobId; this.taskId = taskId; + this.userIdentity = userIdentity; this.labelName = getJobId() + LABEL_SPLITTER + getTaskId(); this.createTimeMs = System.currentTimeMillis(); } @@ -163,35 +161,6 @@ public TRow getTvfInfo(String jobName) { : TimeUtils.longToTimeString(this.getStartTimeMs()))); // load end time trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(this.getFinishTimeMs()))); - - List loadJobs = Env.getCurrentEnv().getLoadManager() - .queryLoadJobsByJobIds(Arrays.asList(this.getTaskId())); - if (!loadJobs.isEmpty()) { - LoadJob loadJob = loadJobs.get(0); - if (loadJob.getLoadingStatus() != null && loadJob.getLoadingStatus().getTrackingUrl() != null) { - trow.addToColumnValue(new TCell().setStringVal(loadJob.getLoadingStatus().getTrackingUrl())); - } else { - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - } - - if (loadJob.getLoadStatistic() != null) { - trow.addToColumnValue(new TCell().setStringVal(loadJob.getLoadStatistic().toJson())); - } else { - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - } - } else { - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - } - - if (this.getUserIdentity() == null) { - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - } else { - trow.addToColumnValue(new TCell().setStringVal(this.getUserIdentity().getQualifiedUser())); - } - trow.addToColumnValue(new TCell().setStringVal("")); - trow.addToColumnValue(new TCell().setStringVal(runningOffset == null - ? FeConstants.null_string : runningOffset.showRange())); return trow; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 32bf44231551a8..c1ca77bc89c6e1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -59,8 +59,8 @@ import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.AlterJobCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.info.BaseViewInfo; -import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertUtils; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; @@ -74,6 +74,7 @@ import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnStateChangeCallback; + import com.google.common.base.Preconditions; import com.google.gson.annotations.SerializedName; import lombok.Getter; @@ -81,6 +82,7 @@ import lombok.extern.log4j.Log4j2; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; + import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; @@ -121,6 +123,7 @@ public class StreamingInsertJob extends AbstractJob createTableIfNotExists() throws UserException, JobException { + private List createTableIfNotExists() throws Exception { List syncTbls = new ArrayList<>(); - List createTableInfos = StreamingJobUtils.generateCreateTableInfos(targetDb, + List createTblCmds = StreamingJobUtils.generateCreateTableCmds(targetDb, dataSourceType, sourceProperties, targetProperties); - for (CreateTableInfo createTableInfo : createTableInfos) { - Env.getCurrentEnv().createTable(createTableInfo); - syncTbls.add(createTableInfo.getTableName()); + for (CreateTableCommand createTblCmd : createTblCmds) { + createTblCmd.run(ConnectContext.get(), null); + syncTbls.add(createTblCmd.getCreateTableInfo().getTableName()); } return syncTbls; } @@ -436,8 +439,8 @@ protected AbstractStreamingTask createStreamingTask() { * @return */ private AbstractStreamingTask createStreamingMultiTblTask() { - return new StreamingMultiTblTask(getJobId(), Env.getCurrentEnv().getNextId(), - dataSourceType, offsetProvider, sourceProperties, targetDb, targetProperties, jobProperties); + return new StreamingMultiTblTask(getJobId(), Env.getCurrentEnv().getNextId(), dataSourceType, + offsetProvider, sourceProperties, targetDb, targetProperties, jobProperties, getCreateUser()); } protected AbstractStreamingTask createStreamingInsertTask() { @@ -591,9 +594,6 @@ public void onRegister() throws JobException { public void onReplayCreate() throws JobException { onRegister(); super.onReplayCreate(); - if (offsetProvider != null) { - this.offsetProvider.replayIfNeed(this); - } } @@ -603,7 +603,12 @@ public void onReplayCreate() throws JobException { * @param replayJob */ public void replayOnUpdated(StreamingInsertJob replayJob) { - setJobStatus(replayJob.getJobStatus()); + if (!JobStatus.RUNNING.equals(replayJob.getJobStatus())) { + // No need to restore in the running state, as scheduling relies on pending states. + // insert TVF does not persist the running state. + // streaming multi task persists the running state when commitOffset() is called. + setJobStatus(replayJob.getJobStatus()); + } try { modifyPropertiesInternal(replayJob.getProperties()); // When the pause state is restarted, it also needs to be updated @@ -614,6 +619,9 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { // should not happen log.error("replay modify streaming insert job properties failed, job id: {}", getJobId(), e); } + if (replayJob.getOffsetProviderPersist() != null) { + setOffsetProviderPersist(replayJob.getOffsetProviderPersist()); + } setExecuteSql(replayJob.getExecuteSql()); setSucceedTaskCount(replayJob.getSucceedTaskCount()); setFailedTaskCount(replayJob.getFailedTaskCount()); @@ -1032,6 +1040,10 @@ public void commitOffset(CommitOffsetRequest offsetRequest) throws JobException } writeLock(); try { + if (offsetRequest.getScannedRows() == 0 && offsetRequest.getScannedBytes() == 0) { + JdbcSourceOffsetProvider op = (JdbcSourceOffsetProvider) offsetProvider; + op.setHasMoreData(false); + } updateJobStatisticAndOffset(offsetRequest); if (this.runningStreamTask != null && this.runningStreamTask instanceof StreamingMultiTblTask) { @@ -1054,4 +1066,10 @@ private void persistOffsetProviderIfNeed() { logUpdateOperation(); } } + + public void replayOffsetProviderIfNeed() throws JobException { + if (this.offsetProviderPersist != null && offsetProvider != null) { + offsetProvider.replayIfNeed(this); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java index 0a97c2f80b7535..635c639256c276 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.Env; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.Status; import org.apache.doris.common.util.Util; import org.apache.doris.job.base.Job; @@ -26,6 +27,7 @@ import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.InsertTask; import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.parser.NereidsParser; @@ -33,11 +35,15 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.QueryState; import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.thrift.TCell; +import org.apache.doris.thrift.TRow; import org.apache.doris.thrift.TStatusCode; import lombok.Getter; import lombok.extern.log4j.Log4j2; +import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.Optional; @@ -61,9 +67,8 @@ public StreamingInsertTask(long jobId, StreamingJobProperties jobProperties, Map originTvfProps, UserIdentity userIdentity) { - super(jobId, taskId); + super(jobId, taskId, userIdentity); this.sql = sql; - this.userIdentity = userIdentity; this.currentDb = currentDb; this.offsetProvider = offsetProvider; this.jobProperties = jobProperties; @@ -170,4 +175,38 @@ public void closeOrReleaseResources() { ctx = null; } } + + @Override + public TRow getTvfInfo(String jobName) { + TRow trow = super.getTvfInfo(jobName); + List loadJobs = Env.getCurrentEnv().getLoadManager() + .queryLoadJobsByJobIds(Arrays.asList(this.getTaskId())); + if (!loadJobs.isEmpty()) { + LoadJob loadJob = loadJobs.get(0); + if (loadJob.getLoadingStatus() != null && loadJob.getLoadingStatus().getTrackingUrl() != null) { + trow.addToColumnValue(new TCell().setStringVal(loadJob.getLoadingStatus().getTrackingUrl())); + } else { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } + + if (loadJob.getLoadStatistic() != null) { + trow.addToColumnValue(new TCell().setStringVal(loadJob.getLoadStatistic().toJson())); + } else { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } + } else { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } + + if (this.getUserIdentity() == null) { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } else { + trow.addToColumnValue(new TCell().setStringVal(this.getUserIdentity().getQualifiedUser())); + } + trow.addToColumnValue(new TCell().setStringVal("")); + trow.addToColumnValue(new TCell().setStringVal(runningOffset == null + ? FeConstants.null_string : runningOffset.showRange())); + return trow; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java index b549195211cd36..d0034de4b4a266 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -65,7 +65,7 @@ private void handlePendingState() throws JobException { return; } } - + streamingInsertJob.replayOffsetProviderIfNeed(); streamingInsertJob.createStreamingTask(); streamingInsertJob.updateJobStatus(JobStatus.RUNNING); streamingInsertJob.setAutoResumeCount(0); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java index c47aa4b4e31d38..52b4100afe7ee4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java @@ -17,7 +17,9 @@ package org.apache.doris.job.extensions.insert.streaming; +import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.Env; +import org.apache.doris.common.FeConstants; import org.apache.doris.httpv2.entity.ResponseBody; import org.apache.doris.httpv2.rest.RestApiStatusCode; import org.apache.doris.httpv2.rest.StreamingJobAction.CommitOffsetRequest; @@ -35,14 +37,18 @@ import org.apache.doris.proto.InternalService.PRequestCdcClientResult; import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.system.Backend; +import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TRow; import org.apache.doris.thrift.TStatusCode; + import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import lombok.Getter; import lombok.extern.log4j.Log4j2; + import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -69,8 +75,9 @@ public StreamingMultiTblTask(Long jobId, Map sourceProperties, String targetDb, Map targetProperties, - StreamingJobProperties jobProperties) { - super(jobId, taskId); + StreamingJobProperties jobProperties, + UserIdentity userIdentity) { + super(jobId, taskId, userIdentity); this.dataSourceType = dataSourceType; this.offsetProvider = offsetProvider; this.sourceProperties = sourceProperties; @@ -185,7 +192,9 @@ public void successCallback(CommitOffsetRequest offsetRequest) { this.status = TaskStatus.SUCCESS; this.finishTimeMs = System.currentTimeMillis(); JdbcOffset runOffset = (JdbcOffset) this.runningOffset; - + if (!isCallable()) { + return; + } // set end offset to running offset Map offsetMeta; try { @@ -209,9 +218,8 @@ public void successCallback(CommitOffsetRequest offsetRequest) { + " offset commit request split id {}", runOffset.getSplit().getSplitId(), splitId); throw new RuntimeException("Split id is not consistent"); } - if (!isCallable()) { - return; - } + this.scannedRows = offsetRequest.getScannedRows(); + this.scannedBytes = offsetRequest.getScannedBytes(); Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); if (null == job) { log.info("job is null, job id is {}", jobId); @@ -241,4 +249,24 @@ public boolean isTimeout() { // todo: need to config return (System.currentTimeMillis() - createTimeMs) > 300 * 1000; } + + @Override + public TRow getTvfInfo(String jobName) { + TRow trow = super.getTvfInfo(jobName); + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + Map statistic = new HashMap<>(); + statistic.put("scannedRows", scannedRows); + statistic.put("loadBytes", scannedBytes); + trow.addToColumnValue(new TCell().setStringVal(new Gson().toJson(statistic))); + + if (this.getUserIdentity() == null) { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } else { + trow.addToColumnValue(new TCell().setStringVal(this.getUserIdentity().getQualifiedUser())); + } + trow.addToColumnValue(new TCell().setStringVal("")); + trow.addToColumnValue(new TCell().setStringVal(runningOffset == null + ? FeConstants.null_string : runningOffset.showRange())); + return trow; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index ccd0e422f82199..7eabbccf300b82 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -17,6 +17,7 @@ package org.apache.doris.job.offset; +import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; @@ -102,7 +103,7 @@ public interface SourceOffsetProvider { * * @return */ - default void replayIfNeed(StreamingInsertJob job) { + default void replayIfNeed(StreamingInsertJob job) throws JobException { } default String getPersistInfo() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java index a71faf4e60d019..e3eb37c716c8c0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -20,6 +20,7 @@ import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.jdbc.split.AbstractSourceSplit; import org.apache.doris.job.offset.jdbc.split.BinlogSplit; + import com.google.gson.Gson; import lombok.AllArgsConstructor; import lombok.Getter; @@ -27,6 +28,7 @@ import lombok.Setter; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.MapUtils; + import java.util.HashMap; @Getter @@ -57,7 +59,7 @@ public String showRange() { if (split.snapshotSplit()) { // need to show hw return new Gson().toJson(split); - }else { + } else { BinlogSplit binlogSplit = (BinlogSplit) split; HashMap showMap = new HashMap<>(); showMap.put(JdbcSourceOffsetProvider.SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index 2a3cac19c21985..657a4c7ca49a94 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -37,6 +37,7 @@ import org.apache.doris.system.Backend; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TStatusCode; + import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -47,10 +48,10 @@ import lombok.extern.log4j.Log4j2; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.MapUtils; + import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -79,6 +80,8 @@ public class JdbcSourceOffsetProvider implements SourceOffsetProvider { @SerializedName("bop") Map binlogOffsetPersist; + boolean hasMoreData = true; + public JdbcSourceOffsetProvider(Long jobId, DataSourceType sourceType, Map sourceProperties) { this.jobId = jobId; this.sourceType = sourceType; @@ -207,6 +210,10 @@ public void fetchRemoteMeta(Map properties) throws Exception { new TypeReference>>() { } ); + if (endBinlogOffset != null + && !endBinlogOffset.equals(responseObj.getData())) { + hasMoreData = true; + } endBinlogOffset = responseObj.getData(); } catch (ExecutionException | InterruptedException | IOException ex) { log.error("Get end offset error: ", ex); @@ -216,6 +223,10 @@ public void fetchRemoteMeta(Map properties) throws Exception { @Override public boolean hasMoreDataToConsume() { + if (!hasMoreData) { + return false; + } + if (currentOffset == null) { return true; } @@ -310,25 +321,28 @@ public Offset deserializeOffsetProperty(String offset) { * Replay snapshot splits if needed */ @Override - public void replayIfNeed(StreamingInsertJob job) { - if (checkNeedSplitChunks(job.getSourceProperties())) { + public void replayIfNeed(StreamingInsertJob job) throws JobException { + String offsetProviderPersist = job.getOffsetProviderPersist(); + if (job.getOffsetProviderPersist() == null) { + return; + } + JdbcSourceOffsetProvider replayFromPersist = GsonUtils.GSON.fromJson(offsetProviderPersist, + JdbcSourceOffsetProvider.class); + this.binlogOffsetPersist = replayFromPersist.getBinlogOffsetPersist(); + this.chunkHighWatermarkMap = replayFromPersist.getChunkHighWatermarkMap(); + + if (MapUtils.isNotEmpty(binlogOffsetPersist)) { + currentOffset = new JdbcOffset(); + currentOffset.setSplit(new BinlogSplit(binlogOffsetPersist)); + } else { try { Map> snapshotSplits = StreamingJobUtils.restoreSplitsToJob(job.getJobId()); - String offsetProviderPersist = job.getOffsetProviderPersist(); - JdbcSourceOffsetProvider replayFromPersist = GsonUtils.GSON.fromJson(offsetProviderPersist, - JdbcSourceOffsetProvider.class); - // need a flag to check pure binlog? - this.chunkHighWatermarkMap = replayFromPersist.getChunkHighWatermarkMap(); if (MapUtils.isNotEmpty(chunkHighWatermarkMap) && MapUtils.isNotEmpty(snapshotSplits)) { - replaySnapshotSplits(chunkHighWatermarkMap, snapshotSplits); - } - this.binlogOffsetPersist = replayFromPersist.getBinlogOffsetPersist(); - if (MapUtils.isNotEmpty(binlogOffsetPersist)) { - currentOffset = new JdbcOffset(); - currentOffset.setSplit(new BinlogSplit(binlogOffsetPersist)); + recalculateRemainingSplits(chunkHighWatermarkMap, snapshotSplits); } } catch (Exception ex) { - log.error("Failed to restore splits for job {}", job.getJobId(), ex); + log.warn("Replay snapshot splits error with job {} ", job.getJobId(), ex); + throw new JobException(ex); } } } @@ -337,7 +351,7 @@ public void replayIfNeed(StreamingInsertJob job) { * Assign the HW value to the synchronized Split, * and remove the Split from remainSplit and place it in finishedSplit. */ - private void replaySnapshotSplits(Map>> chunkHighWatermarkMap, + private void recalculateRemainingSplits(Map>> chunkHighWatermarkMap, Map> snapshotSplits) { if (this.finishedSplits == null) { this.finishedSplits = new ArrayList<>(); @@ -349,23 +363,26 @@ private void replaySnapshotSplits(Map>> if (MapUtils.isEmpty(splitIdToHighWatermark)) { continue; } - - List tableSplits = snapshotSplits.get(tableId); + // db.schema.table + String tableName = getTableName(tableId); + if (tableName == null) { + continue; + } + List tableSplits = snapshotSplits.get(tableName); if (CollectionUtils.isEmpty(tableSplits)) { continue; } - for (Iterator iterator = tableSplits.iterator(); iterator.hasNext(); ) { - SnapshotSplit split = iterator.next(); + tableSplits.removeIf(split -> { String splitId = split.getSplitId(); - Map highWatermark = splitIdToHighWatermark.get(splitId); if (highWatermark != null) { split.setHighWatermark(highWatermark); finishedSplits.add(split); - iterator.remove(); + return true; } - } + return false; + }); } this.remainingSplits = snapshotSplits.values().stream() @@ -373,6 +390,14 @@ private void replaySnapshotSplits(Map>> .collect(Collectors.toList()); } + private String getTableName(String tableId) { + if (tableId == null) { + return null; + } + String[] split = tableId.split("\\."); + return split[split.length - 1]; + } + @Override public String getPersistInfo() { return GsonUtils.GSON.toJson(this); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java index a1bb22d54a0965..1494e47416f7c2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java @@ -21,6 +21,7 @@ import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.Setter; + import java.util.List; import java.util.Map; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java index 695008ba51bd14..109f99a6688d6e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java @@ -32,6 +32,7 @@ import org.apache.doris.job.common.LoadConstants; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.offset.jdbc.split.SnapshotSplit; +import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; import org.apache.doris.nereids.trees.plans.commands.info.DistributionDescriptor; @@ -39,18 +40,20 @@ import org.apache.doris.nereids.types.DataType; import org.apache.doris.qe.AutoCloseConnectContext; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.SessionVariable; import org.apache.doris.qe.StmtExecutor; import org.apache.doris.statistics.ResultRow; import org.apache.doris.system.Backend; import org.apache.doris.system.BeSelectionPolicy; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TUniqueId; + import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import lombok.extern.log4j.Log4j2; import org.apache.commons.text.StringSubstitutor; + +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -107,11 +110,12 @@ public static void createMetaTableIfNotExist() throws Exception { } } - public static Map> restoreSplitsToJob(Long jobId) throws Exception { + public static Map> restoreSplitsToJob(Long jobId) throws IOException { List resultRows = new ArrayList<>(); String sql = String.format(SELECT_SPLITS_TABLE_TEMPLATE, jobId); - try (AutoCloseConnectContext r = buildConnectContext()) { - StmtExecutor stmtExecutor = new StmtExecutor(r.connectContext, sql); + try (AutoCloseConnectContext context + = new AutoCloseConnectContext(buildConnectContext())) { + StmtExecutor stmtExecutor = new StmtExecutor(context.connectContext, sql); resultRows = stmtExecutor.executeInternalQuery(); } @@ -119,7 +123,8 @@ public static Map> restoreSplitsToJob(Long jobId) th for (ResultRow row : resultRows) { String tableName = row.get(0); String chunkListStr = row.get(1); - List splits = Arrays.asList(objectMapper.readValue(chunkListStr, SnapshotSplit[].class)); + List splits = + new ArrayList<>(Arrays.asList(objectMapper.readValue(chunkListStr, SnapshotSplit[].class))); tableSplits.put(tableName, splits); } return tableSplits; @@ -157,27 +162,25 @@ private static void batchInsert(List values) throws Exception { } private static void executeInsert(String sql) throws Exception { - try (AutoCloseConnectContext r = buildConnectContext()) { - StmtExecutor stmtExecutor = new StmtExecutor(r.connectContext, sql); + try (AutoCloseConnectContext context + = new AutoCloseConnectContext(buildConnectContext())) { + StmtExecutor stmtExecutor = new StmtExecutor(context.connectContext, sql); stmtExecutor.execute(); } } - private static AutoCloseConnectContext buildConnectContext() { - ConnectContext connectContext = new ConnectContext(); - connectContext.getState().setInternal(true); - SessionVariable sessionVariable = connectContext.getSessionVariable(); - sessionVariable.setEnableInsertStrict(true); - sessionVariable.setInsertMaxFilterRatio(1); - sessionVariable.enableProfile = false; - connectContext.setEnv(Env.getCurrentEnv()); - connectContext.setDatabase(FeConstants.INTERNAL_DB_NAME); + private static ConnectContext buildConnectContext() { + ConnectContext ctx = new ConnectContext(); + ctx.setEnv(Env.getCurrentEnv()); + ctx.setCurrentUserIdentity(UserIdentity.ADMIN); + ctx.getState().reset(); + ctx.getState().setInternal(true); + ctx.getState().setNereids(true); + ctx.setThreadLocalInfo(); UUID uuid = UUID.randomUUID(); TUniqueId queryId = new TUniqueId(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits()); - connectContext.setQueryId(queryId); - connectContext.setStartTime(); - connectContext.setCurrentUserIdentity(UserIdentity.ADMIN); - return new AutoCloseConnectContext(connectContext); + ctx.setQueryId(queryId); + return ctx; } private static JdbcClient getJdbcClient(DataSourceType sourceType, Map properties) @@ -219,10 +222,10 @@ public static Backend selectBackend(Long jobId) throws JobException { return backend; } - public static List generateCreateTableInfos(String targetDb, DataSourceType sourceType, + public static List generateCreateTableCmds(String targetDb, DataSourceType sourceType, Map properties, Map targetProperties) throws JobException { - List createtblInfos = new ArrayList<>(); + List createtblCmds = new ArrayList<>(); String includeTables = properties.get(LoadConstants.INCLUDE_TABLES); String excludeTables = properties.get(LoadConstants.EXCLUDE_TABLES); List includeTablesList = new ArrayList<>(); @@ -251,9 +254,7 @@ public static List generateCreateTableInfos(String targetDb, Da } List columns = jdbcClient.getColumnsFromJdbc(database, table); - - List primaryKeys = columns.stream().filter(Column::isKey).map(Column::getName) - .collect(Collectors.toList()); + List primaryKeys = jdbcClient.getPrimaryKeys(database, table); if (primaryKeys.isEmpty()) { primaryKeys.add(columns.get(0).getName()); log.info("table {} no primary key, use first column {} to primary key", table, @@ -262,7 +263,7 @@ public static List generateCreateTableInfos(String targetDb, Da // Convert Column to ColumnDefinition List columnDefinitions = columns.stream().map(col -> { DataType dataType = DataType.fromCatalogType(col.getType()); - return new ColumnDefinition(col.getName(), dataType, col.isKey(), col.isAllowNull(), col.getComment()); + return new ColumnDefinition(col.getName(), dataType, col.isAllowNull(), col.getComment()); }).collect(Collectors.toList()); // Create DistributionDescriptor @@ -294,10 +295,10 @@ public static List generateCreateTableInfos(String targetDb, Da ImmutableMap.of(), // extProperties ImmutableList.of() // clusterKeyColumnNames ); - createtblInfo.analyzeEngine(); - createtblInfos.add(createtblInfo); + CreateTableCommand createtblCmd = new CreateTableCommand(Optional.empty(), createtblInfo); + createtblCmds.add(createtblCmd); } - return createtblInfos; + return createtblCmds; } private static Map getTableCreateProperties(Map properties) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java index 98cdb9014a5539..d88a3c4a5d08e1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java @@ -144,10 +144,6 @@ public ColumnDefinition(String name, DataType type, boolean isNullable, String c this(name, type, false, null, isNullable, Optional.empty(), comment); } - public ColumnDefinition(String name, DataType type, boolean isKey, boolean isNullable, String comment) { - this(name, type, isKey, null, isNullable, Optional.empty(), comment); - } - public String getName() { return name; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java index 9541479e44fc6f..dc7c125d6d48c0 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java @@ -1,7 +1,23 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.cdcclient.config; import org.apache.doris.cdcclient.common.Env; - import jakarta.annotation.PostConstruct; import org.springframework.beans.factory.annotation.Value; import org.springframework.stereotype.Component; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java index 30b47a69b38754..756e323df462a7 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package org.apache.doris.cdcclient.config; import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchRecordReq.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchRecordReq.java index 6decb34171cac8..3fcc474c076f48 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchRecordReq.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchRecordReq.java @@ -23,7 +23,7 @@ @Data @EqualsAndHashCode(callSuper = true) public class FetchRecordReq extends JobBaseRecordReq { - private boolean reload; + private boolean reload = true; private int fetchSize; @Override diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index acc4a5b31c5b31..418d0edcc2f8f5 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -25,16 +25,12 @@ import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; import org.apache.doris.cdcclient.source.reader.SourceReader; import org.apache.doris.cdcclient.source.reader.SplitReadResult; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import io.debezium.data.Envelope; + import org.apache.commons.collections.CollectionUtils; import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.source.SourceRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.stereotype.Component; + import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -45,6 +41,13 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import io.debezium.data.Envelope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Component; + /** Pipeline coordinator. */ @Component public class PipelineCoordinator { @@ -167,8 +170,22 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { } } if (!hasData) { - // should not happen, No data will be distributed task - throw new StreamLoadException("No data to write"); + // todo: need return the lastest heartbeat offset, means the maximum offset that the + // current job can recover. + if (readBinlog) { + // BinlogSplit binlogSplit = + // OBJECT_MAPPER.convertValue(writeRecordReq.getMeta(), + // BinlogSplit.class); + // Map metaResp = new + // HashMap<>(binlogSplit.getStartingOffset()); + // metaResp.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); + Map offsetRes = + sourceReader.extractBinlogOffset(readResult.getSplit()); + batchStreamLoad.commitOffset(offsetRes, scannedRows, scannedBytes); + return; + } else { + throw new RuntimeException("should not happen"); + } } // wait all stream load finish @@ -187,7 +204,6 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { } // request fe api batchStreamLoad.commitOffset(metaResponse, scannedRows, scannedBytes); - // batchStreamLoad.commitTransaction(metaResponse); } finally { sourceReader.finishSplitRecords(); if (batchStreamLoad != null) { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java index 5926817ecdb816..67efff55c0a016 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java @@ -70,7 +70,7 @@ public List deserialize(Map context, SourceRecord record LOG.debug("Ignored schema change record: {}", record); return Collections.emptyList(); } else { - LOG.trace("Ignored other record: {}", record); + LOG.info("Ignored other record: {}", record); return Collections.emptyList(); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index cb586823d5d928..a0c1bdb82ac1cb 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -55,7 +55,7 @@ default SplitReadResult readSplitRecords(JobBaseRecordReq bas Map extractSnapshotOffset(Object splitState, Object split); /** Extract offset information from binlog split. */ - Map extractBinlogOffset(Object split, boolean pureBinlogPhase); + Map extractBinlogOffset(Object split); /** * Get split ID from the split. This method should be implemented by each SourceReader to handle diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index be96f90cbdd929..6848900f08809b 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -33,16 +33,7 @@ import org.apache.doris.cdcclient.source.split.BinlogSplit; import org.apache.doris.cdcclient.source.split.SnapshotSplit; import org.apache.doris.cdcclient.utils.ConfigUtil; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.github.shyiko.mysql.binlog.BinaryLogClient; -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.connector.mysql.MySqlPartition; -import io.debezium.document.Array; -import io.debezium.relational.Column; -import io.debezium.relational.TableId; -import io.debezium.relational.history.HistoryRecord; -import io.debezium.relational.history.TableChanges; + import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.utils.Preconditions; @@ -51,12 +42,12 @@ import org.apache.flink.cdc.connectors.mysql.debezium.reader.DebeziumReader; import org.apache.flink.cdc.connectors.mysql.debezium.reader.SnapshotSplitReader; import org.apache.flink.cdc.connectors.mysql.debezium.task.context.StatefulTaskContext; -import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; import org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlSnapshotSplitAssigner; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; import org.apache.flink.cdc.connectors.mysql.source.split.FinishedSnapshotSplitInfo; import org.apache.flink.cdc.connectors.mysql.source.split.MySqlBinlogSplit; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlBinlogSplitState; import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSnapshotSplit; import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSnapshotSplitState; import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplit; @@ -69,8 +60,7 @@ import org.apache.flink.cdc.debezium.history.FlinkJsonTableChangeSerializer; import org.apache.flink.table.types.logical.RowType; import org.apache.kafka.connect.source.SourceRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + import java.io.IOException; import java.sql.SQLException; import java.util.ArrayList; @@ -83,6 +73,23 @@ import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; +import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlPartition; +import io.debezium.document.Array; +import io.debezium.relational.Column; +import io.debezium.relational.TableId; +import io.debezium.relational.history.HistoryRecord; +import io.debezium.relational.history.TableChanges; +import lombok.Getter; +import lombok.Setter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + public class MySqlSourceReader implements SourceReader { private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceReader.class); private static ObjectMapper objectMapper = new ObjectMapper(); @@ -171,10 +178,7 @@ public SplitReadResult readSplitRecords(JobBaseReco if (currentSplitRecords == null) { DebeziumReader currentReader = jobRuntimeContext.getCurrentReader(); - if (currentReader instanceof BinlogSplitReader) { - // only for binlog reader - currentSplitRecords = pollSplitRecordsWithCurrentReader(currentReader); - } else if (currentReader == null) { + if (currentReader == null) { // build split Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); split = splitFlag.f0; @@ -182,28 +186,34 @@ public SplitReadResult readSplitRecords(JobBaseReco readBinlog = !split.isSnapshotSplit(); currentSplitRecords = pollSplitRecordsWithSplit(split, baseReq); jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); + jobRuntimeContext.setCurrentSplit(split); + } else if (baseReq.isReload()) { + // When a client requests a reload, the split is reconstructed and the binlog reader + // is reset. + Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); + split = splitFlag.f0; + pureBinlogPhase = splitFlag.f1; + // reset binlog reader + closeBinlogReader(); + currentSplitRecords = pollSplitRecordsWithSplit(split, baseReq); + jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); + jobRuntimeContext.setCurrentSplit(split); + } else if (currentReader instanceof BinlogSplitReader) { + // only for binlog reader + currentSplitRecords = pollSplitRecordsWithCurrentReader(currentReader); + split = jobRuntimeContext.getCurrentSplit(); } else { throw new RuntimeException("Should not happen"); } } - // When a client requests a reload, the split is reconstructed and the binlog reader is - // reset. - if (baseReq.isReload() && split == null) { - Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); - split = splitFlag.f0; - pureBinlogPhase = splitFlag.f1; - // reset binlog reader - closeBinlogReader(); - currentSplitRecords = pollSplitRecordsWithSplit(split, baseReq); - jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); - } - // build response with iterator SplitReadResult result = new SplitReadResult<>(); MySqlSplitState currentSplitState = null; if (!readBinlog) { currentSplitState = new MySqlSnapshotSplitState(split.asSnapshotSplit()); + } else { + currentSplitState = new MySqlBinlogSplitState(split.asBinlogSplit()); } Iterator filteredIterator = @@ -240,7 +250,6 @@ private RecordWithMeta buildRecordResponse( Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); if (readBinlog) { lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); - // lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); recordResponse.setMeta(lastMeta); } if (count >= fetchRecord.getFetchSize()) { @@ -261,14 +270,8 @@ private RecordWithMeta buildRecordResponse( } if (CollectionUtils.isEmpty(recordResponse.getRecords())) { if (readBinlog) { - BinlogSplit binlogSplit = - objectMapper.convertValue(fetchRecord.getMeta(), BinlogSplit.class); - Map offsetRes = binlogSplit.getStartingOffset(); - if (split != null) { - offsetRes = split.asBinlogSplit().getStartingOffset().getOffset(); - } - offsetRes.put(SPLIT_ID, BINLOG_SPLIT_ID); - // offsetRes.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); + // + Map offsetRes = extractBinlogOffset(readResult.getSplit()); recordResponse.setMeta(offsetRes); } else { SnapshotSplit snapshotSplit = @@ -318,7 +321,6 @@ private Tuple2 createMySqlSplit( private MySqlSnapshotSplit createSnapshotSplit(Map offset, JobConfig jobConfig) throws JsonProcessingException { - // SnapshotSplit snapshotSplit = SnapshotSplit.fromMap(offset); SnapshotSplit snapshotSplit = objectMapper.convertValue(offset, SnapshotSplit.class); TableId tableId = TableId.parse(snapshotSplit.getTableId()); Object[] splitStart = snapshotSplit.getSplitStart(); @@ -345,7 +347,7 @@ private MySqlSnapshotSplit createSnapshotSplit(Map offset, JobCo } private Tuple2 createBinlogSplit( - Map meta, JobConfig config) throws JsonProcessingException { + Map meta, JobConfig config) { MySqlSourceConfig sourceConfig = getSourceConfig(config); BinlogOffset offsetConfig = null; if (sourceConfig.getStartupOptions() != null) { @@ -356,12 +358,6 @@ private Tuple2 createBinlogSplit( BinlogOffset minOffsetFinishSplits = null; BinlogOffset maxOffsetFinishSplits = null; if (CollectionUtils.isNotEmpty(binlogSplit.getFinishedSplits())) { - // if (meta.containsKey(FINISH_SPLITS)) { // List - // Construct binlogsplit based on the finished split - // String finishSplitsOffset = meta.remove(FINISH_SPLITS); - // List splitWithHW = - // objectMapper.readValue( - // finishSplitsOffset, new TypeReference>() {}); List splitWithHW = binlogSplit.getFinishedSplits(); List assignedSplitLists = splitWithHW.stream() @@ -471,6 +467,7 @@ private SplitRecords pollSplitRecordsWithSplit(MySqlSplit split, JobConfig jobCo currentReader.submitSplit(split); currentSplitId = split.splitId(); // make split record available + // todo: Until debezium_heartbeat is consumed Thread.sleep(1000); dataIt = currentReader.pollSplitRecords(); if (currentReader instanceof SnapshotSplitReader) { @@ -571,7 +568,7 @@ public Map extractSnapshotOffset(Object splitState, Object split } @Override - public Map extractBinlogOffset(Object split, boolean pureBinlogPhase) { + public Map extractBinlogOffset(Object split) { if (split == null) { return null; } @@ -653,52 +650,15 @@ private Map discoverTableSchemas(JobConfig co } } + @Getter + @Setter private static final class JobRuntimeContext { private SnapshotSplitReader snapshotReader; private BinlogSplitReader binlogReader; private DebeziumReader currentReader; private Map tableSchemas; private SplitRecords currentSplitRecords; - - private SnapshotSplitReader getSnapshotReader() { - return snapshotReader; - } - - private void setSnapshotReader(SnapshotSplitReader snapshotReader) { - this.snapshotReader = snapshotReader; - } - - private BinlogSplitReader getBinlogReader() { - return binlogReader; - } - - private void setBinlogReader(BinlogSplitReader binlogReader) { - this.binlogReader = binlogReader; - } - - private DebeziumReader getCurrentReader() { - return currentReader; - } - - private void setCurrentReader(DebeziumReader currentReader) { - this.currentReader = currentReader; - } - - private SplitRecords getCurrentSplitRecords() { - return currentSplitRecords; - } - - private void setCurrentSplitRecords(SplitRecords currentSplitRecords) { - this.currentSplitRecords = currentSplitRecords; - } - - private Map getTableSchemas() { - return tableSchemas; - } - - private void setTableSchemas(Map tableSchemas) { - this.tableSchemas = tableSchemas; - } + private MySqlSplit currentSplit; private void close() { if (snapshotReader != null) { @@ -746,11 +706,16 @@ public boolean hasNext() { SourceRecord element = sourceIterator.next(); if (RecordUtils.isWatermarkEvent(element)) { BinlogOffset watermark = RecordUtils.getWatermark(element); - if (RecordUtils.isHighWatermarkEvent(element) && splitState != null) { + if (RecordUtils.isHighWatermarkEvent(element) + && splitState.isSnapshotSplitState()) { splitState.asSnapshotSplitState().setHighWatermark(watermark); } } else if (RecordUtils.isHeartbeatEvent(element)) { LOG.debug("Receive heartbeat event: {}", element); + if (splitState.isBinlogSplitState()) { + BinlogOffset position = RecordUtils.getBinlogPosition(element); + splitState.asBinlogSplitState().setStartingOffset(position); + } } else if (RecordUtils.isDataChangeRecord(element)) { nextRecord = element; return true; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java index 1fdf1353d02b8d..b56bed007a6f89 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java @@ -22,17 +22,25 @@ import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.NoArgsConstructor; import lombok.Setter; @Getter @Setter @EqualsAndHashCode(callSuper = true) -@NoArgsConstructor public class BinlogSplit extends AbstractSourceSplit { private static final long serialVersionUID = 1L; + public static final String BINLOG_SPLIT_ID = "binlog-split"; private Map startingOffset; private Map endingOffset; // binlog split meta, first binlog split requires private List finishedSplits; + + public BinlogSplit() { + this.splitId = BINLOG_SPLIT_ID; + } + + public BinlogSplit(Map startingOffset) { + this.splitId = BINLOG_SPLIT_ID; + this.startingOffset = startingOffset; + } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java index 5750bb8f61c95b..01bf7f0d7e67ef 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java @@ -17,14 +17,9 @@ package org.apache.doris.cdcclient.source.split; -import org.apache.flink.cdc.common.utils.Preconditions; - import java.util.List; import java.util.Map; -import java.util.Optional; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import lombok.EqualsAndHashCode; import lombok.Getter; @@ -58,48 +53,4 @@ public SnapshotSplit( this.splitEnd = splitEnd; this.highWatermark = highWatermark; } - - public static SnapshotSplit fromMap(Map map) throws JsonProcessingException { - if (map == null || map.isEmpty()) { - return null; - } - - SnapshotSplit split = new SnapshotSplit(); - String splitId = map.get("splitId"); - String tableId = map.get("tableId"); - String splitKeyStr = map.get("splitKey"); - Preconditions.checkNotNull(splitKeyStr, "splitKey must not be null"); - List splitKey = - objectMapper.readValue(splitKeyStr, new TypeReference>() {}); - - split.setSplitId(splitId); - split.setTableId(tableId); - split.setSplitKey(splitKey); - - String splitStartStr = map.get("splitStart"); - if (splitStartStr != null) { - Object[] splitStart = objectMapper.readValue(splitStartStr, Object[].class); - split.setSplitStart(splitStart); - } - - String splitEndStr = map.get("splitEnd"); - if (splitEndStr != null) { - Object[] splitEnd = objectMapper.readValue(splitEndStr, Object[].class); - split.setSplitEnd(splitEnd); - } - - String highWatermarkStr = map.get("highWatermark"); - if (highWatermarkStr != null) { - Map highWatermark = - objectMapper.readValue( - highWatermarkStr, new TypeReference>() {}); - split.setHighWatermark(highWatermark); - } - - return split; - } - - public static String getOrEmptyArray(Map map, String key) { - return Optional.ofNullable(map.get(key)).orElse("[]"); - } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index a2ef64dda2dcbb..26929925bd7b07 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -20,13 +20,17 @@ import org.apache.doris.cdcclient.constants.LoadConstants; import org.apache.doris.cdcclient.model.JobConfig; import com.mysql.cj.conf.ConnectionUrl; +import io.debezium.connector.mysql.MySqlConnection; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetBuilder; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetUtils; import org.apache.flink.cdc.connectors.mysql.table.StartupOptions; +import java.sql.SQLException; import java.util.Map; import java.util.Properties; @@ -65,13 +69,21 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { } // setting startMode - String startupMode = cdcConfig.get(MySqlSourceOptions.SCAN_STARTUP_MODE.key()); + String startupMode = cdcConfig.get(LoadConstants.STARTUP_MODE); if ("initial".equalsIgnoreCase(startupMode)) { - configFactory.startupOptions(StartupOptions.initial()); - } else if ("earliest-offset".equalsIgnoreCase(startupMode)) { + // do not need set offset when initial + // configFactory.startupOptions(StartupOptions.initial()); + } else if ("earliest".equalsIgnoreCase(startupMode)) { configFactory.startupOptions(StartupOptions.earliest()); - } else if ("latest-offset".equalsIgnoreCase(startupMode)) { + BinlogOffset binlogOffset = + initializeEffectiveOffset( + configFactory, StartupOptions.earliest().binlogOffset); + configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); + } else if ("latest".equalsIgnoreCase(startupMode)) { configFactory.startupOptions(StartupOptions.latest()); + BinlogOffset binlogOffset = + initializeEffectiveOffset(configFactory, StartupOptions.latest().binlogOffset); + configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); } else if ("specific-offset".equalsIgnoreCase(startupMode)) { BinlogOffsetBuilder offsetBuilder = BinlogOffset.builder(); String file = cdcConfig.get(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_FILE.key()); @@ -110,17 +122,34 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { Long ts = Long.parseLong( cdcConfig.get(MySqlSourceOptions.SCAN_STARTUP_TIMESTAMP_MILLIS.key())); - configFactory.startupOptions(StartupOptions.timestamp(ts)); + BinlogOffset binlogOffset = + initializeEffectiveOffset( + configFactory, StartupOptions.timestamp(ts).binlogOffset); + configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); + } else { + throw new RuntimeException("Unknown startup_mode " + startupMode); } Properties jdbcProperteis = new Properties(); jdbcProperteis.putAll(cu.getOriginalProperties()); configFactory.jdbcProperties(jdbcProperteis); + // configFactory.heartbeatInterval(Duration.ofMillis(1)); + if (cdcConfig.containsKey(LoadConstants.SPLIT_SIZE)) { configFactory.splitSize(Integer.parseInt(cdcConfig.get(LoadConstants.SPLIT_SIZE))); } return configFactory.createConfig(0); } + + private static BinlogOffset initializeEffectiveOffset( + MySqlSourceConfigFactory configFactory, BinlogOffset binlogOffset) { + MySqlSourceConfig config = configFactory.createConfig(0); + try (MySqlConnection connection = DebeziumUtils.createMySqlConnection(config)) { + return BinlogOffsetUtils.initializeEffectiveOffset(binlogOffset, connection, config); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } } From 4f3a1e58624c4aaee2216bfaed25851835423e22 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 10 Dec 2025 15:46:10 +0800 Subject: [PATCH 16/27] add case for mysql sync --- .../thirdparties/docker-compose/mysql/my.cnf | 3 + .../datasource/jdbc/client/JdbcClient.java | 3 + .../jdbc/client/JdbcMySQLClient.java | 21 ++++ .../insert/streaming/StreamingInsertJob.java | 39 ++++-- .../streaming/StreamingJobStatistic.java | 3 + .../offset/jdbc/JdbcSourceOffsetProvider.java | 16 ++- .../doris/job/util/StreamingJobUtils.java | 21 ++-- .../cdcclient/config/BackendPortHolder.java | 1 + .../controller/ClientController.java | 9 +- .../reader/mysql/MySqlSourceReader.java | 6 +- .../doris/cdcclient/utils/ConfigUtil.java | 7 +- .../cdc/test_streaming_mysql_job.out | 11 ++ .../cdc/test_streaming_mysql_job.groovy | 119 ++++++++++++++++++ 13 files changed, 231 insertions(+), 28 deletions(-) create mode 100644 regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy diff --git a/docker/thirdparties/docker-compose/mysql/my.cnf b/docker/thirdparties/docker-compose/mysql/my.cnf index a6ef77e8f1910e..1f9664e8afc2b5 100644 --- a/docker/thirdparties/docker-compose/mysql/my.cnf +++ b/docker/thirdparties/docker-compose/mysql/my.cnf @@ -17,6 +17,9 @@ character-set-server=utf8 collation-server=utf8_bin max_connections=1000 +log-bin=mysql-bin +binlog_format=ROW +server-id=1 [client] default-character-set=utf8 [mysql] diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java index 0669b7c91099b5..bae2304e9b31a2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java @@ -404,6 +404,9 @@ public List getColumnsFromJdbc(String remoteDbName, String remoteTableNa return dorisTableSchema; } + /** + * get primary keys of one table + */ public List getPrimaryKeys(String remoteDbName, String remoteTableName) { Connection conn = getConnection(); ResultSet rs = null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java index 0c18ebaf7b7709..98ebd66dd14675 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java @@ -185,6 +185,27 @@ public List getJdbcColumnsInfo(String remoteDbName, String remo return tableSchema; } + @Override + public List getPrimaryKeys(String remoteDbName, String remoteTableName) { + Connection conn = getConnection(); + ResultSet rs = null; + List primaryKeys = Lists.newArrayList(); + try { + DatabaseMetaData databaseMetaData = conn.getMetaData(); + rs = databaseMetaData.getPrimaryKeys(remoteDbName, null, remoteTableName); + while (rs.next()) { + String fieldName = rs.getString("COLUMN_NAME"); + primaryKeys.add(fieldName); + } + } catch (SQLException e) { + throw new JdbcClientException("failed to get jdbc primary key info for remote table `%s.%s`: %s", + remoteDbName, remoteTableName, Util.getRootCauseMessage(e)); + } finally { + close(rs, conn); + } + return primaryKeys; + } + protected String getCatalogName(Connection conn) throws SQLException { return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index c1ca77bc89c6e1..08e7f48015897a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -18,6 +18,7 @@ package org.apache.doris.job.extensions.insert.streaming; import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; import org.apache.doris.cloud.proto.Cloud; import org.apache.doris.cloud.rpc.MetaServiceProxy; @@ -74,7 +75,6 @@ import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnStateChangeCallback; - import com.google.common.base.Preconditions; import com.google.gson.annotations.SerializedName; import lombok.Getter; @@ -82,7 +82,6 @@ import lombok.extern.log4j.Log4j2; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; - import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; @@ -100,7 +99,13 @@ public class StreamingInsertJob extends AbstractJob> implements TxnStateChangeCallback, GsonPostProcessable { private long dbId; + // Streaming job statistics, all persisted in txn attachment private StreamingJobStatistic jobStatistic = new StreamingJobStatistic(); + // Non-txn persisted statistics, used for streaming multi task + @Getter + @Setter + @SerializedName("ntjs") + private StreamingJobStatistic nonTxnJobStatistic = new StreamingJobStatistic(); @Getter @Setter @SerializedName("fr") @@ -233,8 +238,12 @@ private List createTableIfNotExists() throws Exception { List syncTbls = new ArrayList<>(); List createTblCmds = StreamingJobUtils.generateCreateTableCmds(targetDb, dataSourceType, sourceProperties, targetProperties); + Database db = Env.getCurrentEnv().getInternalCatalog().getDbNullable(targetDb); + Preconditions.checkNotNull(db, "target database %s does not exist", targetDb); for (CreateTableCommand createTblCmd : createTblCmds) { - createTblCmd.run(ConnectContext.get(), null); + if (!db.isTableExist(createTblCmd.getCreateTableInfo().getTableName())) { + createTblCmd.run(ConnectContext.get(), null); + } syncTbls.add(createTblCmd.getCreateTableInfo().getTableName()); } return syncTbls; @@ -585,6 +594,15 @@ private void updateJobStatisticAndOffset(CommitOffsetRequest offsetRequest) { offsetProvider.updateOffset(offsetProvider.deserializeOffset(offsetRequest.getOffset())); } + private void updateNoTxnJobStatisticAndOffset(CommitOffsetRequest offsetRequest) { + if (this.nonTxnJobStatistic == null) { + this.nonTxnJobStatistic = new StreamingJobStatistic(); + } + this.nonTxnJobStatistic.setScannedRows(this.nonTxnJobStatistic.getScannedRows() + offsetRequest.getScannedRows()); + this.nonTxnJobStatistic.setLoadBytes(this.nonTxnJobStatistic.getLoadBytes() + offsetRequest.getScannedBytes()); + offsetProvider.updateOffset(offsetProvider.deserializeOffset(offsetRequest.getOffset())); + } + @Override public void onRegister() throws JobException { Env.getCurrentGlobalTransactionMgr().getCallbackFactory().addCallback(this); @@ -622,6 +640,9 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { if (replayJob.getOffsetProviderPersist() != null) { setOffsetProviderPersist(replayJob.getOffsetProviderPersist()); } + if (replayJob.getNonTxnJobStatistic() != null){ + setNonTxnJobStatistic(replayJob.getNonTxnJobStatistic()); + } setExecuteSql(replayJob.getExecuteSql()); setSucceedTaskCount(replayJob.getSucceedTaskCount()); setFailedTaskCount(replayJob.getFailedTaskCount()); @@ -708,9 +729,13 @@ public TRow getTvfInfo() { } else { trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); } - - trow.addToColumnValue(new TCell().setStringVal( - jobStatistic == null ? FeConstants.null_string : jobStatistic.toJson())); + if (tvfType != null) { + trow.addToColumnValue(new TCell().setStringVal( + jobStatistic == null ? FeConstants.null_string : jobStatistic.toJson())); + } else { + trow.addToColumnValue(new TCell().setStringVal( + nonTxnJobStatistic == null ? FeConstants.null_string : nonTxnJobStatistic.toJson())); + } trow.addToColumnValue(new TCell().setStringVal(failureReason == null ? FeConstants.null_string : failureReason.getMsg())); trow.addToColumnValue(new TCell().setStringVal(jobRuntimeMsg == null @@ -1044,7 +1069,7 @@ public void commitOffset(CommitOffsetRequest offsetRequest) throws JobException JdbcSourceOffsetProvider op = (JdbcSourceOffsetProvider) offsetProvider; op.setHasMoreData(false); } - updateJobStatisticAndOffset(offsetRequest); + updateNoTxnJobStatisticAndOffset(offsetRequest); if (this.runningStreamTask != null && this.runningStreamTask instanceof StreamingMultiTblTask) { if (this.runningStreamTask.getTaskId() != offsetRequest.getTaskId()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java index 8b209cd6e81bf4..71bf9e6f065307 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java @@ -19,15 +19,18 @@ import com.google.gson.Gson; +import com.google.gson.annotations.SerializedName; import lombok.Getter; import lombok.Setter; public class StreamingJobStatistic { @Getter @Setter + @SerializedName("scannedRows") private long scannedRows; @Getter @Setter + @SerializedName("loadBytes") private long loadBytes; @Getter @Setter diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index 657a4c7ca49a94..c315bdfb1dd327 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -52,6 +52,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -338,7 +339,11 @@ public void replayIfNeed(StreamingInsertJob job) throws JobException { try { Map> snapshotSplits = StreamingJobUtils.restoreSplitsToJob(job.getJobId()); if (MapUtils.isNotEmpty(chunkHighWatermarkMap) && MapUtils.isNotEmpty(snapshotSplits)) { - recalculateRemainingSplits(chunkHighWatermarkMap, snapshotSplits); + SnapshotSplit lastSnapshotSplit = recalculateRemainingSplits(chunkHighWatermarkMap, snapshotSplits); + if (this.remainingSplits.isEmpty()) { + currentOffset = new JdbcOffset(); + currentOffset.setSplit(lastSnapshotSplit); + } } } catch (Exception ex) { log.warn("Replay snapshot splits error with job {} ", job.getJobId(), ex); @@ -351,12 +356,12 @@ public void replayIfNeed(StreamingInsertJob job) throws JobException { * Assign the HW value to the synchronized Split, * and remove the Split from remainSplit and place it in finishedSplit. */ - private void recalculateRemainingSplits(Map>> chunkHighWatermarkMap, + private SnapshotSplit recalculateRemainingSplits(Map>> chunkHighWatermarkMap, Map> snapshotSplits) { if (this.finishedSplits == null) { this.finishedSplits = new ArrayList<>(); } - + SnapshotSplit lastSnapshotSplit = null; for (Map.Entry>> entry : chunkHighWatermarkMap.entrySet()) { String tableId = entry.getKey(); Map> splitIdToHighWatermark = entry.getValue(); @@ -372,7 +377,7 @@ private void recalculateRemainingSplits(Map { String splitId = split.getSplitId(); Map highWatermark = splitIdToHighWatermark.get(splitId); @@ -388,6 +393,7 @@ private void recalculateRemainingSplits(Map createTbls) throws JobException { // todo: When splitting takes a long time, it needs to be changed to asynchronous. if (checkNeedSplitChunks(sourceProperties)) { - Map> tableSplits = new HashMap<>(); + Map> tableSplits = new LinkedHashMap<>(); for (String tbl : createTbls) { List snapshotSplits = requestTableSplits(tbl); tableSplits.put(tbl, snapshotSplits); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java index 109f99a6688d6e..c71eb284652361 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java @@ -57,6 +57,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -69,14 +70,14 @@ public class StreamingJobUtils { public static final String FULL_QUALIFIED_META_TBL_NAME = InternalCatalog.INTERNAL_CATALOG_NAME + "." + FeConstants.INTERNAL_DB_NAME + "." + INTERNAL_STREAMING_JOB_META_TABLE_NAME; private static final String CREATE_META_TABLE = "CREATE TABLE %s(\n" - + "id varchar(32),\n" - + "job_id varchar(256),\n" + + "id int,\n" + + "job_id bigint,\n" + "table_name string,\n" + "chunk_list json\n" + ")\n" - + "UNIQUE KEY(id)\n" - + "DISTRIBUTED BY HASH(id)\n" - + "BUCKETS 1\n" + + "UNIQUE KEY(id, job_id)\n" + + "DISTRIBUTED BY HASH(job_id)\n" + + "BUCKETS 2\n" + "PROPERTIES ('replication_num' = '1')"; // todo: modify replication num like statistic sys tbl private static final String BATCH_INSERT_INTO_META_TABLE_TEMPLATE = "INSERT INTO " + FULL_QUALIFIED_META_TBL_NAME + " values"; @@ -85,7 +86,7 @@ public class StreamingJobUtils { "('${id}', '${job_id}', '${table_name}', '${chunk_list}')"; private static final String SELECT_SPLITS_TABLE_TEMPLATE = - "SELECT table_name, chunk_list from " + FULL_QUALIFIED_META_TBL_NAME + " WHERE job_id='%s'"; + "SELECT table_name, chunk_list from " + FULL_QUALIFIED_META_TBL_NAME + " WHERE job_id='%s' ORDER BY id ASC"; private static final ObjectMapper objectMapper = new ObjectMapper(); @@ -111,7 +112,7 @@ public static void createMetaTableIfNotExist() throws Exception { } public static Map> restoreSplitsToJob(Long jobId) throws IOException { - List resultRows = new ArrayList<>(); + List resultRows; String sql = String.format(SELECT_SPLITS_TABLE_TEMPLATE, jobId); try (AutoCloseConnectContext context = new AutoCloseConnectContext(buildConnectContext())) { @@ -119,7 +120,7 @@ public static Map> restoreSplitsToJob(Long jobId) th resultRows = stmtExecutor.executeInternalQuery(); } - Map> tableSplits = new HashMap<>(); + Map> tableSplits = new LinkedHashMap<>(); for (ResultRow row : resultRows) { String tableName = row.get(0); String chunkListStr = row.get(1); @@ -132,15 +133,17 @@ public static Map> restoreSplitsToJob(Long jobId) th public static void insertSplitsToMeta(Long jobId, Map> tableSplits) throws Exception { List values = new ArrayList<>(); + int index = 1; for (Map.Entry> entry : tableSplits.entrySet()) { Map params = new HashMap<>(); - params.put("id", UUID.randomUUID().toString().replace("-", "")); + params.put("id", index + ""); params.put("job_id", jobId + ""); params.put("table_name", entry.getKey()); params.put("chunk_list", objectMapper.writeValueAsString(entry.getValue())); StringSubstitutor stringSubstitutor = new StringSubstitutor(params); String sql = stringSubstitutor.replace(INSERT_INTO_META_TABLE_TEMPLATE); values.add(sql); + index++; } batchInsert(values); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java index dc7c125d6d48c0..c19771345305a6 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java @@ -18,6 +18,7 @@ package org.apache.doris.cdcclient.config; import org.apache.doris.cdcclient.common.Env; + import jakarta.annotation.PostConstruct; import org.springframework.beans.factory.annotation.Value; import org.springframework.stereotype.Component; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index fe6f6ffce5afe6..c2bb71219d0681 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -26,9 +26,6 @@ import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; import org.apache.doris.cdcclient.service.PipelineCoordinator; import org.apache.doris.cdcclient.source.reader.SourceReader; - -import java.util.List; - import jakarta.servlet.http.HttpServletRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +35,7 @@ import org.springframework.web.bind.annotation.RequestMapping; import org.springframework.web.bind.annotation.RequestMethod; import org.springframework.web.bind.annotation.RestController; +import java.util.List; @RestController public class ClientController { @@ -73,6 +71,11 @@ public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { /** Fetch records from source reader and Write records to backend */ @RequestMapping(path = "/api/writeRecords", method = RequestMethod.POST) public Object writeRecord(@RequestBody WriteRecordReq recordReq) { + LOG.info( + "Received write record request for jobId={}, taskId={}, meta={}", + recordReq.getJobId(), + recordReq.getTaskId(), + recordReq.getMeta()); pipelineCoordinator.writeRecordsAsync(recordReq); return ResponseEntityBuilder.ok("Request accepted, processing asynchronously"); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index 6848900f08809b..42506b0464b02e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -168,6 +168,7 @@ public SplitReadResult readSplitRecords(JobBaseReco if (offsetMeta == null || offsetMeta.isEmpty()) { throw new RuntimeException("miss meta offset"); } + LOG.info("Job {} read split records with offset: {}", baseReq.getJobId(), offsetMeta); // If there is an active split being consumed, reuse it directly; // Otherwise, create a new snapshot/binlog split based on offset and start the reader. @@ -210,10 +211,11 @@ public SplitReadResult readSplitRecords(JobBaseReco // build response with iterator SplitReadResult result = new SplitReadResult<>(); MySqlSplitState currentSplitState = null; + MySqlSplit currentSplit = jobRuntimeContext.getCurrentSplit(); if (!readBinlog) { - currentSplitState = new MySqlSnapshotSplitState(split.asSnapshotSplit()); + currentSplitState = new MySqlSnapshotSplitState(currentSplit.asSnapshotSplit()); } else { - currentSplitState = new MySqlBinlogSplitState(split.asBinlogSplit()); + currentSplitState = new MySqlBinlogSplitState(currentSplit.asBinlogSplit()); } Iterator filteredIterator = diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index 26929925bd7b07..d8e3fc37297122 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -19,8 +19,7 @@ import org.apache.doris.cdcclient.constants.LoadConstants; import org.apache.doris.cdcclient.model.JobConfig; -import com.mysql.cj.conf.ConnectionUrl; -import io.debezium.connector.mysql.MySqlConnection; + import org.apache.commons.lang3.StringUtils; import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; @@ -30,10 +29,14 @@ import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetBuilder; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetUtils; import org.apache.flink.cdc.connectors.mysql.table.StartupOptions; + import java.sql.SQLException; import java.util.Map; import java.util.Properties; +import com.mysql.cj.conf.ConnectionUrl; +import io.debezium.connector.mysql.MySqlConnection; + public class ConfigUtil { public static String getServerId(long jobId) { diff --git a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out new file mode 100644 index 00000000000000..566412e8f743ed --- /dev/null +++ b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out @@ -0,0 +1,11 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +A 1 +B 2 +C 3 + +-- !select -- +A 1 +B 10 +Doris 18 + diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy new file mode 100644 index 00000000000000..9367ebfd7c9789 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy @@ -0,0 +1,119 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job", "p0,external,mysql,external_docker,external_docker_mysql") { + def jobName = "test_streaming_mysql_job_name" + def currentDb = (sql "select database()")[0][0] + def tableName = "user_info" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${tableName} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // create test + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${tableName}""" + sql """ + CREATE TABLE ${mysqlDb}.${tableName} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB; + """ + // mock snapshot data + sql """INSERT INTO ${mysqlDb}.${tableName} (name, age) VALUES ('A', 1);""" + sql """INSERT INTO ${mysqlDb}.${tableName} (name, age) VALUES ('B', 2);""" + sql """INSERT INTO ${mysqlDb}.${tableName} (name, age) VALUES ('C', 3);""" + } + + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${tableName}", + "startup_mode" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + // check table created + def showTables = sql """ show tables from ${currentDb} like '${tableName}'; """ + assert showTables.size() == 1 + + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + // check job status and succeed task count larger than 1 + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + // check snapshot data + qt_select """ SELECT * FROM ${tableName} order by name asc """ + + // mock mysql incremental into + + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """INSERT INTO ${mysqlDb}.${tableName} (name,age) VALUES ('Doris',18);""" + sql """UPDATE ${mysqlDb}.${tableName} SET age = 10 WHERE name = 'B';""" + sql """DELETE FROM ${mysqlDb}.${tableName} WHERE name = 'C';""" + } + + sleep(10000); // wait for cdc incremental data + + // check incremental data + qt_select """ SELECT * FROM ${tableName} order by name asc """ + + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } +} From f63e9848026d9ee0406e8bbc934b7ce6558c0baa Mon Sep 17 00:00:00 2001 From: JNSimba <676366545@qq.com> Date: Wed, 10 Dec 2025 16:34:03 +0800 Subject: [PATCH 17/27] code style --- be/src/runtime/cdc_client_manager.cpp | 43 +++++++++---------- be/src/runtime/cdc_client_manager.h | 11 ++--- be/src/runtime/exec_env_init.cpp | 2 +- be/src/service/internal_service.cpp | 8 ++-- be/src/service/internal_service.h | 6 +-- .../insert/streaming/StreamingInsertJob.java | 7 ++- .../offset/jdbc/JdbcSourceOffsetProvider.java | 3 +- fs_brokers/cdc_client/pom.xml | 19 ++++++++ .../src/main/resources/application.properties | 17 ++++++++ 9 files changed, 76 insertions(+), 40 deletions(-) diff --git a/be/src/runtime/cdc_client_manager.cpp b/be/src/runtime/cdc_client_manager.cpp index 8ec1c0381ccaa3..08d128eac32162 100644 --- a/be/src/runtime/cdc_client_manager.cpp +++ b/be/src/runtime/cdc_client_manager.cpp @@ -47,9 +47,9 @@ void handle_sigchld(int sig_no) { // Check CDC client health Status check_cdc_client_health(int retry_times, int sleep_time, std::string& health_response) { - const std::string cdc_health_url = "http://127.0.0.1:" + - std::to_string(doris::config::cdc_client_port) + - "/actuator/health"; + const std::string cdc_health_url = + "http://127.0.0.1:" + std::to_string(doris::config::cdc_client_port) + + "/actuator/health"; auto health_request = [cdc_health_url, &health_response](HttpClient* client) { RETURN_IF_ERROR(client->init(cdc_health_url)); @@ -69,7 +69,7 @@ Status check_cdc_client_health(int retry_times, int sleep_time, std::string& hea if (!is_up) { return Status::InternalError(fmt::format("CDC client unhealthy: url={}, response={}", - cdc_health_url, health_response)); + cdc_health_url, health_response)); } return Status::OK(); @@ -78,7 +78,7 @@ Status check_cdc_client_health(int retry_times, int sleep_time, std::string& hea // Start CDC client process Status start_cdc_client(PRequestCdcClientResult* result) { Status st = Status::OK(); - + // Check DORIS_HOME environment variable const char* doris_home = getenv("DORIS_HOME"); if (!doris_home) { @@ -88,7 +88,7 @@ Status start_cdc_client(PRequestCdcClientResult* result) { } return st; } - + // Check LOG_DIR environment variable const char* log_dir = getenv("LOG_DIR"); if (!log_dir) { @@ -98,12 +98,14 @@ Status start_cdc_client(PRequestCdcClientResult* result) { } return st; } - + const std::string cdc_jar_path = std::string(doris_home) + "/lib/cdc_client/cdc-client.jar"; - const std::string cdc_jar_port = "--server.port=" + std::to_string(doris::config::cdc_client_port); - const std::string backend_http_port = "--backend.http.port=" + std::to_string(config::webserver_port); + const std::string cdc_jar_port = + "--server.port=" + std::to_string(doris::config::cdc_client_port); + const std::string backend_http_port = + "--backend.http.port=" + std::to_string(config::webserver_port); const std::string java_opts = "-Dlog.path=" + std::string(log_dir); - + // check cdc jar exists struct stat buffer; if (stat(cdc_jar_path.c_str(), &buffer) != 0) { @@ -156,13 +158,13 @@ Status start_cdc_client(PRequestCdcClientResult* result) { prctl(PR_SET_PDEATHSIG, SIGKILL); #endif - LOG(INFO) << "Cdc client child process ready to start, " << pid << ", response=" - << std::endl; + LOG(INFO) << "Cdc client child process ready to start, " << pid + << ", response=" << std::endl; std::cout << "Cdc client child process ready to start." << std::endl; std::string java_bin = path + "/bin/java"; // java -jar -Dlog.path=xx cdc-client.jar --server.port=9096 --backend.http.port=8040 execlp(java_bin.c_str(), "java", java_opts.c_str(), "-jar", cdc_jar_path.c_str(), - cdc_jar_port.c_str(), backend_http_port.c_str(),(char*)NULL); + cdc_jar_port.c_str(), backend_http_port.c_str(), (char*)NULL); std::cerr << "Cdc client child process error." << std::endl; exit(1); } else { @@ -198,8 +200,8 @@ void CdcClientManager::stop() { } void CdcClientManager::request_cdc_client_impl(const PRequestCdcClientRequest* request, - PRequestCdcClientResult* result, - google::protobuf::Closure* done) { + PRequestCdcClientResult* result, + google::protobuf::Closure* done) { VLOG_RPC << "request to cdc client, api " << request->api(); brpc::ClosureGuard closure_guard(done); @@ -217,13 +219,11 @@ void CdcClientManager::request_cdc_client_impl(const PRequestCdcClientRequest* r st.to_protobuf(result->mutable_status()); } - Status CdcClientManager::send_request_to_cdc_client(const std::string& api, - const std::string& params_body, - std::string* response) { - std::string remote_url_prefix = fmt::format("http://127.0.0.1:{}{}", - doris::config::cdc_client_port, - api); + const std::string& params_body, + std::string* response) { + std::string remote_url_prefix = + fmt::format("http://127.0.0.1:{}{}", doris::config::cdc_client_port, api); auto cdc_request = [&remote_url_prefix, response, ¶ms_body](HttpClient* client) { RETURN_IF_ERROR(client->init(remote_url_prefix)); @@ -241,4 +241,3 @@ Status CdcClientManager::send_request_to_cdc_client(const std::string& api, } } // namespace doris - diff --git a/be/src/runtime/cdc_client_manager.h b/be/src/runtime/cdc_client_manager.h index 9c97a6a5e6b6e4..cee65245638b03 100644 --- a/be/src/runtime/cdc_client_manager.h +++ b/be/src/runtime/cdc_client_manager.h @@ -37,16 +37,13 @@ class CdcClientManager { void stop(); - // Request CDC client to handle a request + // Request CDC client to handle a request void request_cdc_client_impl(const PRequestCdcClientRequest* request, - PRequestCdcClientResult* result, - google::protobuf::Closure* done); + PRequestCdcClientResult* result, google::protobuf::Closure* done); private: - Status send_request_to_cdc_client(const std::string& api, - const std::string& params_body, - std::string* response); + Status send_request_to_cdc_client(const std::string& api, const std::string& params_body, + std::string* response); }; } // namespace doris - diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 9a0a6b786a027d..c70ee3e663cd5c 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -70,12 +70,12 @@ #include "pipeline/task_scheduler.h" #include "runtime/broker_mgr.h" #include "runtime/cache/result_cache.h" +#include "runtime/cdc_client_manager.h" #include "runtime/client_cache.h" #include "runtime/exec_env.h" #include "runtime/external_scan_context_mgr.h" #include "runtime/fragment_mgr.h" #include "runtime/group_commit_mgr.h" -#include "runtime/cdc_client_manager.h" #include "runtime/heartbeat_flags.h" #include "runtime/index_policy/index_policy_mgr.h" #include "runtime/load_channel_mgr.h" diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index c768273ac497eb..3c072d0b89d998 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -84,6 +84,7 @@ #include "olap/txn_manager.h" #include "olap/wal/wal_manager.h" #include "runtime/cache/result_cache.h" +#include "runtime/cdc_client_manager.h" #include "runtime/descriptors.h" #include "runtime/exec_env.h" #include "runtime/fold_constant_executor.h" @@ -129,7 +130,6 @@ #include "vec/jsonb/serialize.h" #include "vec/runtime/vdata_stream_mgr.h" #include "vec/sink/vmysql_result_writer.h" -#include "runtime/cdc_client_manager.h" namespace google { namespace protobuf { @@ -2407,9 +2407,9 @@ void PInternalService::get_tablet_rowsets(google::protobuf::RpcController* contr } void PInternalService::request_cdc_client(google::protobuf::RpcController* controller, - const PRequestCdcClientRequest* request, - PRequestCdcClientResult* result, - google::protobuf::Closure* done) { + const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) { bool ret = _heavy_work_pool.try_offer([this, request, result, done]() { _exec_env->cdc_client_mgr()->request_cdc_client_impl(request, result, done); }); diff --git a/be/src/service/internal_service.h b/be/src/service/internal_service.h index 792c953d975b20..8535cc2dc98698 100644 --- a/be/src/service/internal_service.h +++ b/be/src/service/internal_service.h @@ -233,9 +233,9 @@ class PInternalService : public PBackendService { google::protobuf::Closure* done) override; void request_cdc_client(google::protobuf::RpcController* controller, - const PRequestCdcClientRequest* request, - PRequestCdcClientResult* result, - google::protobuf::Closure* done) override; + const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) override; private: void _exec_plan_fragment_in_pthread(google::protobuf::RpcController* controller, diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 08e7f48015897a..18d034c33459ab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -75,6 +75,7 @@ import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnStateChangeCallback; + import com.google.common.base.Preconditions; import com.google.gson.annotations.SerializedName; import lombok.Getter; @@ -82,6 +83,7 @@ import lombok.extern.log4j.Log4j2; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; + import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; @@ -598,7 +600,8 @@ private void updateNoTxnJobStatisticAndOffset(CommitOffsetRequest offsetRequest) if (this.nonTxnJobStatistic == null) { this.nonTxnJobStatistic = new StreamingJobStatistic(); } - this.nonTxnJobStatistic.setScannedRows(this.nonTxnJobStatistic.getScannedRows() + offsetRequest.getScannedRows()); + this.nonTxnJobStatistic + .setScannedRows(this.nonTxnJobStatistic.getScannedRows() + offsetRequest.getScannedRows()); this.nonTxnJobStatistic.setLoadBytes(this.nonTxnJobStatistic.getLoadBytes() + offsetRequest.getScannedBytes()); offsetProvider.updateOffset(offsetProvider.deserializeOffset(offsetRequest.getOffset())); } @@ -640,7 +643,7 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { if (replayJob.getOffsetProviderPersist() != null) { setOffsetProviderPersist(replayJob.getOffsetProviderPersist()); } - if (replayJob.getNonTxnJobStatistic() != null){ + if (replayJob.getNonTxnJobStatistic() != null) { setNonTxnJobStatistic(replayJob.getNonTxnJobStatistic()); } setExecuteSql(replayJob.getExecuteSql()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index c315bdfb1dd327..732c9bdae701bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -356,7 +356,8 @@ public void replayIfNeed(StreamingInsertJob job) throws JobException { * Assign the HW value to the synchronized Split, * and remove the Split from remainSplit and place it in finishedSplit. */ - private SnapshotSplit recalculateRemainingSplits(Map>> chunkHighWatermarkMap, + private SnapshotSplit recalculateRemainingSplits( + Map>> chunkHighWatermarkMap, Map> snapshotSplits) { if (this.finishedSplits == null) { this.finishedSplits = new ArrayList<>(); diff --git a/fs_brokers/cdc_client/pom.xml b/fs_brokers/cdc_client/pom.xml index 88a6882105df28..42fcc9ec4e7f97 100644 --- a/fs_brokers/cdc_client/pom.xml +++ b/fs_brokers/cdc_client/pom.xml @@ -1,3 +1,22 @@ + + 4.0.0 diff --git a/fs_brokers/cdc_client/src/main/resources/application.properties b/fs_brokers/cdc_client/src/main/resources/application.properties index 2dc3cb46ef2f49..9a7ee6c3f76ec3 100644 --- a/fs_brokers/cdc_client/src/main/resources/application.properties +++ b/fs_brokers/cdc_client/src/main/resources/application.properties @@ -1,3 +1,20 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ spring.web.resources.add-mappings=false server.port=9096 backend.http.port=8040 \ No newline at end of file From f4e1a9eceb169b2b2201482d22410bc26c1e1db9 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 10 Dec 2025 21:43:19 +0800 Subject: [PATCH 18/27] fix multi table bug --- .../doris/job/common/LoadConstants.java | 7 +- .../insert/streaming/StreamingInsertJob.java | 3 + .../streaming/StreamingMultiTblTask.java | 26 +- .../offset/jdbc/JdbcSourceOffsetProvider.java | 70 ++-- .../doris/job/util/StreamingJobUtils.java | 2 +- fs_brokers/cdc_client/pom.xml | 6 +- .../cdcclient/constants/LoadConstants.java | 8 +- .../controller/ClientController.java | 4 +- .../service/PipelineCoordinator.java | 16 +- .../cdcclient/source/reader/SourceReader.java | 2 + .../source/reader/SplitReadResult.java | 3 - .../reader/mysql/MySqlSourceReader.java | 57 ++-- .../doris/cdcclient/utils/ConfigUtil.java | 120 ++++--- .../mocks/MockSplitEnumeratorContext.java | 320 ++++++++++++++++++ .../src/main/resources/log4j.properties | 2 +- .../cdc/test_streaming_mysql_job.out | 10 +- .../cdc/test_streaming_mysql_job.groovy | 54 +-- 17 files changed, 539 insertions(+), 171 deletions(-) create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/flink/api/connector/source/mocks/MockSplitEnumeratorContext.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java index 3148660d31ef77..8f017462c97a18 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java @@ -26,8 +26,11 @@ public class LoadConstants { public static final String DATABASE = "database"; public static final String INCLUDE_TABLES = "include_tables"; public static final String EXCLUDE_TABLES = "exclude_tables"; - // initial,earliest-offset,latest-offset,specific-offset,timestamp,snapshot - public static final String STARTUP_MODE = "startup_mode"; + // initial,earliest,latest,{binlog,postion},\d{13} + public static final String OFFSET = "offset"; + public static final String OFFSET_INITIAL = "initial"; + public static final String OFFSET_EARLIEST = "earliest"; + public static final String OFFSET_LATEST = "latest"; public static final String SPLIT_SIZE = "split_size"; public static final String TABLE_PROPS_PREFIX = "table.create.properties."; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 18d034c33459ab..e59a9204d8ef2e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -234,6 +234,9 @@ private void checkRequiredSourceProperties() { Preconditions.checkArgument(sourceProperties.get(LoadConstants.INCLUDE_TABLES) != null || sourceProperties.get(LoadConstants.EXCLUDE_TABLES) != null, "Either include_tables or exclude_tables must be specified"); + if (!sourceProperties.containsKey(LoadConstants.OFFSET)) { + sourceProperties.put(LoadConstants.OFFSET, LoadConstants.OFFSET_LATEST); + } } private List createTableIfNotExists() throws Exception { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java index 52b4100afe7ee4..3f73ba47bd3381 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java @@ -49,7 +49,6 @@ import lombok.Getter; import lombok.extern.log4j.Log4j2; -import java.io.IOException; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -128,17 +127,22 @@ private void sendWriteRequest() throws JobException { + result.getResponse()); } String response = result.getResponse(); - ResponseBody responseObj = objectMapper.readValue( - response, - new TypeReference>() { - } - ); - if (responseObj.getCode() == RestApiStatusCode.OK.code) { - log.info("Send write records request successfully, response: {}", responseObj.getData()); - return; + try { + ResponseBody responseObj = objectMapper.readValue( + response, + new TypeReference>() { + } + ); + if (responseObj.getCode() == RestApiStatusCode.OK.code) { + log.info("Send write records request successfully, response: {}", responseObj.getData()); + return; + } + } catch (JsonProcessingException e) { + log.error("Failed to parse write records response: {}", response, e); + throw new JobException("Failed to parse write records response: " + response); } - throw new JobException("Failed to send write records request , error message: " + responseObj); - } catch (ExecutionException | InterruptedException | IOException ex) { + throw new JobException("Failed to send write records request , error message: " + response); + } catch (ExecutionException | InterruptedException ex) { log.error("Send write request failed: ", ex); throw new JobException(ex); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index 732c9bdae701bb..45075f3c3f2871 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -49,7 +49,6 @@ import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.MapUtils; -import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedHashMap; @@ -206,17 +205,22 @@ public void fetchRemoteMeta(Map properties) throws Exception { + result.getResponse()); } String response = result.getResponse(); - ResponseBody> responseObj = objectMapper.readValue( - response, - new TypeReference>>() { - } - ); - if (endBinlogOffset != null - && !endBinlogOffset.equals(responseObj.getData())) { - hasMoreData = true; + try { + ResponseBody> responseObj = objectMapper.readValue( + response, + new TypeReference>>() { + } + ); + if (endBinlogOffset != null + && !endBinlogOffset.equals(responseObj.getData())) { + hasMoreData = true; + } + endBinlogOffset = responseObj.getData(); + } catch (JsonProcessingException e) { + log.error("Failed to parse end offset response: {}", response, e); + throw new JobException("Failed to parse end offset response: " + response); } - endBinlogOffset = responseObj.getData(); - } catch (ExecutionException | InterruptedException | IOException ex) { + } catch (ExecutionException | InterruptedException ex) { log.error("Get end offset error: ", ex); throw new JobException(ex); } @@ -276,13 +280,18 @@ private boolean compareOffset(Map offsetFirst, Map responseObj = objectMapper.readValue( - response, - new TypeReference>() { - } - ); - return responseObj.getData() > 0; - } catch (ExecutionException | InterruptedException | IOException ex) { + try { + ResponseBody responseObj = objectMapper.readValue( + response, + new TypeReference>() { + } + ); + return responseObj.getData() > 0; + } catch (JsonProcessingException e) { + log.error("Failed to parse compare offset response: {}", response, e); + throw new JobException("Failed to parse compare offset response: " + response); + } + } catch (ExecutionException | InterruptedException ex) { log.error("Compare offset error: ", ex); throw new JobException(ex); } @@ -456,14 +465,19 @@ private List requestTableSplits(String table) throws JobException + result.getResponse()); } String response = result.getResponse(); - ResponseBody> responseObj = objectMapper.readValue( - response, - new TypeReference>>() { - } - ); - List splits = responseObj.getData(); - return splits; - } catch (ExecutionException | InterruptedException | IOException ex) { + try { + ResponseBody> responseObj = objectMapper.readValue( + response, + new TypeReference>>() { + } + ); + List splits = responseObj.getData(); + return splits; + } catch (JsonProcessingException e) { + log.error("Failed to parse split response: {}", response, e); + throw new JobException("Failed to parse split response: " + response); + } + } catch (ExecutionException | InterruptedException ex) { log.error("Get splits error: ", ex); throw new JobException(ex); } @@ -492,10 +506,10 @@ private Map buildSplitParams(String table) { } private boolean checkNeedSplitChunks(Map sourceProperties) { - String startMode = sourceProperties.get(LoadConstants.STARTUP_MODE); + String startMode = sourceProperties.get(LoadConstants.OFFSET); if (startMode == null) { return false; } - return "snapshot".equalsIgnoreCase(startMode) || "initial".equalsIgnoreCase(startMode); + return LoadConstants.OFFSET_INITIAL.equalsIgnoreCase(startMode); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java index c71eb284652361..db237598305344 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java @@ -294,7 +294,7 @@ public static List generateCreateTableCmds(String targetDb, PartitionTableInfo.EMPTY, // partitionTableInfo distribution, // distribution ImmutableList.of(), // rollups - tableCreateProperties, // properties + new HashMap<>(tableCreateProperties), // properties ImmutableMap.of(), // extProperties ImmutableList.of() // clusterKeyColumnNames ); diff --git a/fs_brokers/cdc_client/pom.xml b/fs_brokers/cdc_client/pom.xml index 42fcc9ec4e7f97..3e0c1dbb46e888 100644 --- a/fs_brokers/cdc_client/pom.xml +++ b/fs_brokers/cdc_client/pom.xml @@ -127,7 +127,7 @@ under the License. org.apache.flink flink-connector-mysql-cdc - 3.2.1 + 3.5.0 org.apache.flink @@ -258,9 +258,9 @@ under the License. true - ../../fe/check/suppressions.xml + ../../fe/check/checkstyle/suppressions.xml true - ../../fe/check/checkstyle.xml + ../../fe/check/checkstyle/checkstyle.xml true true diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java index 10f3e0b838051e..bfc1d27bc18fc7 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java @@ -26,10 +26,12 @@ public class LoadConstants { public static final String DATABASE = "database"; public static final String INCLUDE_TABLES = "include_tables"; public static final String EXCLUDE_TABLES = "exclude_tables"; - // initial,earliest-offset,latest-offset,specific-offset,timestamp,snapshot - public static final String STARTUP_MODE = "startup_mode"; + // initial,earliest,latest,{binlog,postion},\d{13} + public static final String OFFSET = "offset"; + public static final String OFFSET_INITIAL = "initial"; + public static final String OFFSET_EARLIEST = "earliest"; + public static final String OFFSET_LATEST = "latest"; public static final String SPLIT_SIZE = "split_size"; public static final String TABLE_PROPS_PREFIX = "table.create.properties."; - public static final String DELETE_SIGN_KEY = "__DORIS_DELETE_SIGN__"; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index c2bb71219d0681..f3fddad95c3a73 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -26,6 +26,9 @@ import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; import org.apache.doris.cdcclient.service.PipelineCoordinator; import org.apache.doris.cdcclient.source.reader.SourceReader; + +import java.util.List; + import jakarta.servlet.http.HttpServletRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,7 +38,6 @@ import org.springframework.web.bind.annotation.RequestMapping; import org.springframework.web.bind.annotation.RequestMethod; import org.springframework.web.bind.annotation.RestController; -import java.util.List; @RestController public class ClientController { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 418d0edcc2f8f5..5bd8d186bec56b 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -121,8 +121,6 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { batchStreamLoad.setCurrentTaskId(writeRecordReq.getTaskId()); batchStreamLoad.setFrontendAddress(writeRecordReq.getFrontendAddress()); batchStreamLoad.setToken(writeRecordReq.getToken()); - boolean readBinlog = readResult.isReadBinlog(); - boolean pureBinlogPhase = readResult.isPureBinlogPhase(); boolean hasData = false; // Record start time for maxInterval check @@ -151,9 +149,9 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); - if (readBinlog && sourceReader.getSplitId(readResult.getSplit()) != null) { + if (sourceReader.isBinlogSplit(readResult.getSplit()) + && sourceReader.getSplitId(readResult.getSplit()) != null) { lastMeta.put(SPLIT_ID, sourceReader.getSplitId(readResult.getSplit())); - // lastMeta.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); } metaResponse = lastMeta; } @@ -172,13 +170,7 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { if (!hasData) { // todo: need return the lastest heartbeat offset, means the maximum offset that the // current job can recover. - if (readBinlog) { - // BinlogSplit binlogSplit = - // OBJECT_MAPPER.convertValue(writeRecordReq.getMeta(), - // BinlogSplit.class); - // Map metaResp = new - // HashMap<>(binlogSplit.getStartingOffset()); - // metaResp.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); + if (sourceReader.isBinlogSplit(readResult.getSplit())) { Map offsetRes = sourceReader.extractBinlogOffset(readResult.getSplit()); batchStreamLoad.commitOffset(offsetRes, scannedRows, scannedBytes); @@ -191,7 +183,7 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { // wait all stream load finish batchStreamLoad.forceFlush(); // update offset meta - if (!readBinlog) { + if (!sourceReader.isBinlogSplit(readResult.getSplit())) { Map offsetRes = sourceReader.extractSnapshotOffset( readResult.getSplitState(), readResult.getSplit()); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index a0c1bdb82ac1cb..24863c1e4ace51 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -66,6 +66,8 @@ default SplitReadResult readSplitRecords(JobBaseRecordReq bas */ String getSplitId(Object split); + boolean isBinlogSplit(Object split); + /** Called when closing */ void close(Long jobId); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java index cd96a7e0fc50c5..1b21c3d3818fd5 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java @@ -34,7 +34,4 @@ public class SplitReadResult { private Iterator recordIterator; private SplitState splitState; private Split split; - - private boolean readBinlog; - private boolean pureBinlogPhase; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index 42506b0464b02e..cfef866a1bcc1f 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -35,6 +35,7 @@ import org.apache.doris.cdcclient.utils.ConfigUtil; import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.utils.Preconditions; import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; @@ -172,47 +173,40 @@ public SplitReadResult readSplitRecords(JobBaseReco // If there is an active split being consumed, reuse it directly; // Otherwise, create a new snapshot/binlog split based on offset and start the reader. - boolean readBinlog = true; MySqlSplit split = null; - boolean pureBinlogPhase = true; SplitRecords currentSplitRecords = jobRuntimeContext.getCurrentSplitRecords(); if (currentSplitRecords == null) { DebeziumReader currentReader = jobRuntimeContext.getCurrentReader(); - if (currentReader == null) { + if (currentReader == null || baseReq.isReload()) { + LOG.info( + "No current reader or reload {}, create new split reader", + baseReq.isReload()); // build split Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); split = splitFlag.f0; - pureBinlogPhase = splitFlag.f1; - readBinlog = !split.isSnapshotSplit(); - currentSplitRecords = pollSplitRecordsWithSplit(split, baseReq); - jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); - jobRuntimeContext.setCurrentSplit(split); - } else if (baseReq.isReload()) { - // When a client requests a reload, the split is reconstructed and the binlog reader - // is reset. - Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); - split = splitFlag.f0; - pureBinlogPhase = splitFlag.f1; - // reset binlog reader - closeBinlogReader(); currentSplitRecords = pollSplitRecordsWithSplit(split, baseReq); jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); jobRuntimeContext.setCurrentSplit(split); } else if (currentReader instanceof BinlogSplitReader) { + LOG.info("Continue poll records with current binlog reader"); // only for binlog reader currentSplitRecords = pollSplitRecordsWithCurrentReader(currentReader); split = jobRuntimeContext.getCurrentSplit(); } else { throw new RuntimeException("Should not happen"); } + } else { + LOG.info( + "Continue read records with current split records, splitId: {}", + currentSplitRecords.getSplitId()); } // build response with iterator SplitReadResult result = new SplitReadResult<>(); MySqlSplitState currentSplitState = null; MySqlSplit currentSplit = jobRuntimeContext.getCurrentSplit(); - if (!readBinlog) { + if (currentSplit.isSnapshotSplit()) { currentSplitState = new MySqlSnapshotSplitState(currentSplit.asSnapshotSplit()); } else { currentSplitState = new MySqlBinlogSplitState(currentSplit.asBinlogSplit()); @@ -223,8 +217,6 @@ public SplitReadResult readSplitRecords(JobBaseReco result.setRecordIterator(filteredIterator); result.setSplitState(currentSplitState); - result.setReadBinlog(readBinlog); - result.setPureBinlogPhase(pureBinlogPhase); result.setSplit(split); return result; } @@ -234,8 +226,6 @@ private RecordWithMeta buildRecordResponse( FetchRecordReq fetchRecord, SplitReadResult readResult) throws Exception { RecordWithMeta recordResponse = new RecordWithMeta(); - boolean readBinlog = readResult.isReadBinlog(); - boolean pureBinlogPhase = readResult.isPureBinlogPhase(); MySqlSplit split = readResult.getSplit(); MySqlSplitState currentSplitState = readResult.getSplitState(); int count = 0; @@ -250,7 +240,7 @@ private RecordWithMeta buildRecordResponse( count += serializedRecords.size(); // update meta Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); - if (readBinlog) { + if (split.isBinlogSplit()) { lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); recordResponse.setMeta(lastMeta); } @@ -262,7 +252,7 @@ private RecordWithMeta buildRecordResponse( finishSplitRecords(); // Set meta information - if (!readBinlog && currentSplitState != null) { + if (split.isSnapshotSplit() && currentSplitState != null) { BinlogOffset highWatermark = currentSplitState.asSnapshotSplitState().getHighWatermark(); Map offsetRes = highWatermark.getOffset(); @@ -271,8 +261,7 @@ private RecordWithMeta buildRecordResponse( return recordResponse; } if (CollectionUtils.isEmpty(recordResponse.getRecords())) { - if (readBinlog) { - // + if (split.isBinlogSplit()) { Map offsetRes = extractBinlogOffset(readResult.getSplit()); recordResponse.setMeta(offsetRes); } else { @@ -335,7 +324,7 @@ private MySqlSnapshotSplit createSnapshotSplit(Map offset, JobCo // only support one split key String splitKey = splitKeys.get(0); Column splitColumn = tableChange.getTable().columnWithName(splitKey); - RowType splitType = ChunkUtils.getChunkKeyColumnType(splitColumn); + RowType splitType = ChunkUtils.getChunkKeyColumnType(splitColumn, false); MySqlSnapshotSplit split = new MySqlSnapshotSplit( tableId, @@ -438,7 +427,8 @@ private List startSplitChunks( } List remainingSplits = new ArrayList<>(); MySqlSnapshotSplitAssigner splitAssigner = - new MySqlSnapshotSplitAssigner(sourceConfig, 1, remainingTables, false); + new MySqlSnapshotSplitAssigner( + sourceConfig, 1, remainingTables, false, new MockSplitEnumeratorContext(1)); splitAssigner.open(); while (true) { Optional mySqlSplit = splitAssigner.getNext(); @@ -524,7 +514,7 @@ private BinlogSplitReader getBinlogSplitReader(JobConfig config) { private void closeSnapshotReader() { SnapshotSplitReader reusedSnapshotReader = jobRuntimeContext.getSnapshotReader(); if (reusedSnapshotReader != null) { - LOG.debug( + LOG.info( "Close snapshot reader {}", reusedSnapshotReader.getClass().getCanonicalName()); reusedSnapshotReader.close(); DebeziumReader currentReader = @@ -539,7 +529,7 @@ private void closeSnapshotReader() { private void closeBinlogReader() { BinlogSplitReader reusedBinlogReader = jobRuntimeContext.getBinlogReader(); if (reusedBinlogReader != null) { - LOG.debug("Close binlog reader {}", reusedBinlogReader.getClass().getCanonicalName()); + LOG.info("Close binlog reader {}", reusedBinlogReader.getClass().getCanonicalName()); reusedBinlogReader.close(); DebeziumReader currentReader = jobRuntimeContext.getCurrentReader(); @@ -590,6 +580,15 @@ public String getSplitId(Object split) { return mysqlSplit.splitId(); } + @Override + public boolean isBinlogSplit(Object split) { + if (split == null) { + return false; + } + MySqlSplit mysqlSplit = (MySqlSplit) split; + return mysqlSplit.isBinlogSplit(); + } + @Override public void close(Long jobId) { jobRuntimeContext.close(); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index d8e3fc37297122..959cd1187bff66 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -20,24 +20,29 @@ import org.apache.doris.cdcclient.constants.LoadConstants; import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory; -import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; -import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetBuilder; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetUtils; import org.apache.flink.cdc.connectors.mysql.table.StartupOptions; import java.sql.SQLException; +import java.util.Arrays; import java.util.Map; import java.util.Properties; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import com.mysql.cj.conf.ConnectionUrl; import io.debezium.connector.mysql.MySqlConnection; public class ConfigUtil { + private static final ObjectMapper mapper = new ObjectMapper(); public static String getServerId(long jobId) { return String.valueOf(Math.abs(String.valueOf(jobId).hashCode())); @@ -59,78 +64,60 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { configFactory.includeSchemaChanges(false); - String includingTables = cdcConfig.getOrDefault(LoadConstants.INCLUDE_TABLES, ".*"); - String includingPattern = String.format("(%s)\\.(%s)", databaseName, includingTables); + String includingTables = cdcConfig.get(LoadConstants.INCLUDE_TABLES); + String[] includingTbls = + Arrays.stream(includingTables.split(",")) + .map(t -> databaseName + "." + t.trim()) + .toArray(String[]::new); + configFactory.tableList(includingTbls); + String excludingTables = cdcConfig.get(LoadConstants.EXCLUDE_TABLES); - if (StringUtils.isEmpty(excludingTables)) { - configFactory.tableList(includingPattern); - } else { - String excludingPattern = - String.format("?!(%s\\.(%s))$", databaseName, excludingTables); - String tableList = String.format("(%s)(%s)", excludingPattern, includingPattern); - configFactory.tableList(tableList); + if (StringUtils.isNotEmpty(excludingTables)) { + String excludingTbls = + Arrays.stream(excludingTables.split(",")) + .map(t -> databaseName + "." + t.trim()) + .toString(); + configFactory.excludeTableList(excludingTbls); } // setting startMode - String startupMode = cdcConfig.get(LoadConstants.STARTUP_MODE); - if ("initial".equalsIgnoreCase(startupMode)) { + String startupMode = cdcConfig.get(LoadConstants.OFFSET); + if (LoadConstants.OFFSET_INITIAL.equalsIgnoreCase(startupMode)) { // do not need set offset when initial // configFactory.startupOptions(StartupOptions.initial()); - } else if ("earliest".equalsIgnoreCase(startupMode)) { + } else if (LoadConstants.OFFSET_EARLIEST.equalsIgnoreCase(startupMode)) { configFactory.startupOptions(StartupOptions.earliest()); BinlogOffset binlogOffset = initializeEffectiveOffset( configFactory, StartupOptions.earliest().binlogOffset); configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); - } else if ("latest".equalsIgnoreCase(startupMode)) { + } else if (LoadConstants.OFFSET_LATEST.equalsIgnoreCase(startupMode)) { configFactory.startupOptions(StartupOptions.latest()); BinlogOffset binlogOffset = initializeEffectiveOffset(configFactory, StartupOptions.latest().binlogOffset); configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); - } else if ("specific-offset".equalsIgnoreCase(startupMode)) { - BinlogOffsetBuilder offsetBuilder = BinlogOffset.builder(); - String file = cdcConfig.get(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_FILE.key()); - Long pos = - Long.valueOf( - cdcConfig.get( - MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_POS.key())); - if (file != null && pos != null) { - offsetBuilder.setBinlogFilePosition(file, pos); - } else { - offsetBuilder.setBinlogFilePosition("", 0); - } - - if (cdcConfig.containsKey( - MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_EVENTS.key())) { - long skipEvents = - Long.parseLong( - cdcConfig.getOrDefault( - MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_EVENTS - .key(), - "0")); - offsetBuilder.setSkipEvents(skipEvents); + } else if (isJson(startupMode)) { + // start from specific offset + Map offsetMap = toStringMap(startupMode); + if (MapUtils.isEmpty(offsetMap)) { + throw new RuntimeException("Incorrect offset " + startupMode); } - if (cdcConfig.containsKey( - MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_ROWS.key())) { - long skipRows = - Long.parseLong( - cdcConfig.getOrDefault( - MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_ROWS - .key(), - "0")); - offsetBuilder.setSkipRows(skipRows); + if (offsetMap.containsKey(BinlogOffset.BINLOG_FILENAME_OFFSET_KEY) + && offsetMap.containsKey(BinlogOffset.BINLOG_POSITION_OFFSET_KEY)) { + BinlogOffset binlogOffset = new BinlogOffset(offsetMap); + configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); + } else { + throw new RuntimeException("Incorrect offset " + startupMode); } - configFactory.startupOptions(StartupOptions.specificOffset(offsetBuilder.build())); - } else if ("timestamp".equalsIgnoreCase(startupMode)) { - Long ts = - Long.parseLong( - cdcConfig.get(MySqlSourceOptions.SCAN_STARTUP_TIMESTAMP_MILLIS.key())); + } else if (is13Timestamp(startupMode)) { + // start from timestamp + Long ts = Long.parseLong(startupMode); BinlogOffset binlogOffset = initializeEffectiveOffset( configFactory, StartupOptions.timestamp(ts).binlogOffset); configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); } else { - throw new RuntimeException("Unknown startup_mode " + startupMode); + throw new RuntimeException("Unknown offset " + startupMode); } Properties jdbcProperteis = new Properties(); @@ -138,7 +125,6 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { configFactory.jdbcProperties(jdbcProperteis); // configFactory.heartbeatInterval(Duration.ofMillis(1)); - if (cdcConfig.containsKey(LoadConstants.SPLIT_SIZE)) { configFactory.splitSize(Integer.parseInt(cdcConfig.get(LoadConstants.SPLIT_SIZE))); } @@ -155,4 +141,32 @@ private static BinlogOffset initializeEffectiveOffset( throw new RuntimeException(e); } } + + private static boolean is13Timestamp(String s) { + return s != null && s.matches("\\d{13}"); + } + + private static boolean isJson(String str) { + if (str == null || str.trim().isEmpty()) { + return false; + } + try { + JsonNode node = mapper.readTree(str); + return node.isObject(); + } catch (Exception e) { + return false; + } + } + + private static Map toStringMap(String json) { + if (!isJson(json)) { + return null; + } + + try { + return mapper.readValue(json, new TypeReference>() {}); + } catch (JsonProcessingException e) { + return null; // 或抛异常,按你需要调整 + } + } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/flink/api/connector/source/mocks/MockSplitEnumeratorContext.java b/fs_brokers/cdc_client/src/main/java/org/apache/flink/api/connector/source/mocks/MockSplitEnumeratorContext.java new file mode 100644 index 00000000000000..448e9acce516de --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/flink/api/connector/source/mocks/MockSplitEnumeratorContext.java @@ -0,0 +1,320 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package org.apache.flink.api.connector.source.mocks; + +import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.api.connector.source.SupportsIntermediateNoMoreSplits; +import org.apache.flink.metrics.groups.SplitEnumeratorMetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.util.ThrowableCatchingRunnable; + +import javax.annotation.Nonnull; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; + +/** + * A mock class for {@link SplitEnumeratorContext}. Copy from + * https://github.com/apache/flink/blob/release-1.20.3/flink-core/src/test/java/org/apache/flink/api/connector/source/mocks/MockSplitEnumeratorContext.java + */ +public class MockSplitEnumeratorContext + implements SplitEnumeratorContext, SupportsIntermediateNoMoreSplits, AutoCloseable { + private final Map> sentSourceEvent; + private final ConcurrentMap registeredReaders; + private final List> splitsAssignmentSequence; + private final ExecutorService workerExecutor; + private final ExecutorService mainExecutor; + private final TestingExecutorThreadFactory mainThreadFactory; + private final AtomicReference errorInWorkerThread; + private final AtomicReference errorInMainThread; + private final BlockingQueue>> oneTimeCallables; + private final List>> periodicCallables; + private final AtomicBoolean stoppedAcceptAsyncCalls; + private final boolean[] subtaskHasNoMoreSplits; + + private final int parallelism; + + public MockSplitEnumeratorContext(int parallelism) { + this.sentSourceEvent = new HashMap<>(); + this.registeredReaders = new ConcurrentHashMap<>(); + this.splitsAssignmentSequence = new ArrayList<>(); + this.parallelism = parallelism; + this.errorInWorkerThread = new AtomicReference<>(); + this.errorInMainThread = new AtomicReference<>(); + this.oneTimeCallables = new ArrayBlockingQueue<>(100); + this.periodicCallables = Collections.synchronizedList(new ArrayList<>()); + this.mainThreadFactory = getThreadFactory("SplitEnumerator-main", errorInMainThread); + this.workerExecutor = + getExecutor(getThreadFactory("SplitEnumerator-worker", errorInWorkerThread)); + this.mainExecutor = getExecutor(mainThreadFactory); + this.stoppedAcceptAsyncCalls = new AtomicBoolean(false); + this.subtaskHasNoMoreSplits = new boolean[parallelism]; + } + + @Override + public SplitEnumeratorMetricGroup metricGroup() { + return UnregisteredMetricsGroup.createSplitEnumeratorMetricGroup(); + } + + @Override + public void sendEventToSourceReader(int subtaskId, SourceEvent event) { + try { + if (!mainThreadFactory.isCurrentThreadMainExecutorThread()) { + mainExecutor + .submit( + () -> + sentSourceEvent + .computeIfAbsent(subtaskId, id -> new ArrayList<>()) + .add(event)) + .get(); + } else { + sentSourceEvent.computeIfAbsent(subtaskId, id -> new ArrayList<>()).add(event); + } + } catch (Exception e) { + throw new RuntimeException("Failed to assign splits", e); + } + } + + @Override + public int currentParallelism() { + return parallelism; + } + + @Override + public Map registeredReaders() { + return registeredReaders; + } + + @Override + public void assignSplits(SplitsAssignment newSplitAssignments) { + splitsAssignmentSequence.add(newSplitAssignments); + } + + @Override + public void signalNoMoreSplits(int subtask) { + subtaskHasNoMoreSplits[subtask] = true; + } + + @Override + public void signalIntermediateNoMoreSplits(int subtask) {} + + public void resetNoMoreSplits(int subtask) { + subtaskHasNoMoreSplits[subtask] = false; + } + + @Override + public void callAsync(Callable callable, BiConsumer handler) { + if (stoppedAcceptAsyncCalls.get()) { + return; + } + oneTimeCallables.add( + () -> + workerExecutor.submit( + wrap( + errorInWorkerThread, + () -> { + try { + T result = callable.call(); + mainExecutor + .submit( + wrap( + errorInMainThread, + () -> + handler.accept( + result, + null))) + .get(); + } catch (Throwable t) { + handler.accept(null, t); + } + }))); + } + + @Override + public void callAsync( + Callable callable, + BiConsumer handler, + long initialDelay, + long period) { + if (stoppedAcceptAsyncCalls.get()) { + return; + } + periodicCallables.add( + () -> + workerExecutor.submit( + wrap( + errorInWorkerThread, + () -> { + try { + T result = callable.call(); + mainExecutor + .submit( + wrap( + errorInMainThread, + () -> + handler.accept( + result, + null))) + .get(); + } catch (Throwable t) { + handler.accept(null, t); + } + }))); + } + + @Override + public void runInCoordinatorThread(Runnable runnable) { + mainExecutor.execute(runnable); + } + + @Override + public void setIsProcessingBacklog(boolean isProcessingBacklog) {} + + public void close() throws Exception { + stoppedAcceptAsyncCalls.set(true); + workerExecutor.shutdownNow(); + mainExecutor.shutdownNow(); + } + + // ------------ helper method to manipulate the context ------------- + + public void runNextOneTimeCallable() throws Throwable { + oneTimeCallables.take().call().get(); + checkError(); + } + + public void runPeriodicCallable(int index) throws Throwable { + periodicCallables.get(index).call().get(); + checkError(); + } + + public Map> getSentSourceEvent() throws Exception { + return workerExecutor.submit(() -> new HashMap<>(sentSourceEvent)).get(); + } + + public void registerReader(ReaderInfo readerInfo) { + registeredReaders.put(readerInfo.getSubtaskId(), readerInfo); + } + + public void unregisterReader(int readerId) { + registeredReaders.remove(readerId); + } + + public List>> getPeriodicCallables() { + return periodicCallables; + } + + public BlockingQueue>> getOneTimeCallables() { + return oneTimeCallables; + } + + public List> getSplitsAssignmentSequence() { + return splitsAssignmentSequence; + } + + public boolean hasNoMoreSplits(int subtaskIndex) { + return subtaskHasNoMoreSplits[subtaskIndex]; + } + + // ------------- private helpers ------------- + + private void checkError() throws Throwable { + if (errorInMainThread.get() != null) { + throw errorInMainThread.get(); + } + if (errorInWorkerThread.get() != null) { + throw errorInWorkerThread.get(); + } + } + + private static TestingExecutorThreadFactory getThreadFactory( + String threadName, AtomicReference error) { + return new TestingExecutorThreadFactory(threadName, error); + } + + private static ExecutorService getExecutor(TestingExecutorThreadFactory threadFactory) { + return Executors.newSingleThreadScheduledExecutor(threadFactory); + } + + private static ThrowableCatchingRunnable wrap(AtomicReference error, Runnable r) { + return new ThrowableCatchingRunnable( + t -> { + if (!error.compareAndSet(null, t)) { + error.get().addSuppressed(t); + } + }, + r); + } + + // -------- private class ----------- + + /** A thread factory class that provides some helper methods. */ + public static class TestingExecutorThreadFactory implements ThreadFactory { + private final String coordinatorThreadName; + private final AtomicReference error; + private Thread t; + + TestingExecutorThreadFactory( + String coordinatorThreadName, AtomicReference error) { + this.coordinatorThreadName = coordinatorThreadName; + this.error = error; + this.t = null; + } + + @Override + public Thread newThread(@Nonnull Runnable r) { + if (t != null) { + throw new IllegalStateException( + "Should never happen. This factory should only be used by a " + + "SingleThreadExecutor."); + } + t = new Thread(r, coordinatorThreadName); + t.setUncaughtExceptionHandler( + (t1, e) -> { + if (!error.compareAndSet(null, e)) { + error.get().addSuppressed(e); + } + }); + return t; + } + + boolean isCurrentThreadMainExecutorThread() { + return Thread.currentThread() == t; + } + } +} diff --git a/fs_brokers/cdc_client/src/main/resources/log4j.properties b/fs_brokers/cdc_client/src/main/resources/log4j.properties index f4c8faf2ff6130..1ac680d4ddc5a2 100644 --- a/fs_brokers/cdc_client/src/main/resources/log4j.properties +++ b/fs_brokers/cdc_client/src/main/resources/log4j.properties @@ -16,7 +16,7 @@ # limitations under the License. ################################################################################ -log4j.rootLogger=INFO, STDOUT, FILE +log4j.rootLogger=INFO, FILE log.path=./logs diff --git a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out index 566412e8f743ed..4bb6ad2eb2fe4f 100644 --- a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out +++ b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out @@ -1,11 +1,11 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -A 1 -B 2 -C 3 +A1 1 +B1 2 -- !select -- -A 1 -B 10 + +-- !select -- +B1 10 Doris 18 diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy index 9367ebfd7c9789..41bca762a8e679 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy @@ -23,11 +23,13 @@ import static java.util.concurrent.TimeUnit.SECONDS suite("test_streaming_mysql_job", "p0,external,mysql,external_docker,external_docker_mysql") { def jobName = "test_streaming_mysql_job_name" def currentDb = (sql "select database()")[0][0] - def tableName = "user_info" + def table1 = "user_info1" + def table2 = "user_info2" def mysqlDb = "test_cdc_db" - sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" - sql """drop table if exists ${currentDb}.${tableName} force""" + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + sql """drop table if exists ${currentDb}.${table2} force""" String enabled = context.config.otherConfigs.get("enableJdbcTest") if (enabled != null && enabled.equalsIgnoreCase("true")) { @@ -40,18 +42,23 @@ suite("test_streaming_mysql_job", "p0,external,mysql,external_docker,external_do // create test connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" - sql """DROP TABLE IF EXISTS ${mysqlDb}.${tableName}""" - sql """ - CREATE TABLE ${mysqlDb}.${tableName} ( + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table2}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( `name` varchar(200) NOT NULL, `age` int DEFAULT NULL, PRIMARY KEY (`name`) - ) ENGINE=InnoDB; - """ + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('A1', 1);""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('B1', 2);""" + sql """CREATE TABLE ${mysqlDb}.${table2} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB""" // mock snapshot data - sql """INSERT INTO ${mysqlDb}.${tableName} (name, age) VALUES ('A', 1);""" - sql """INSERT INTO ${mysqlDb}.${tableName} (name, age) VALUES ('B', 2);""" - sql """INSERT INTO ${mysqlDb}.${tableName} (name, age) VALUES ('C', 3);""" + sql """INSERT INTO ${mysqlDb}.${table2} (name, age) VALUES ('A2', 1);""" + sql """INSERT INTO ${mysqlDb}.${table2} (name, age) VALUES ('B2', 2);""" } sql """CREATE JOB ${jobName} @@ -63,16 +70,18 @@ suite("test_streaming_mysql_job", "p0,external,mysql,external_docker,external_do "user" = "root", "password" = "123456", "database" = "${mysqlDb}", - "include_tables" = "${tableName}", - "startup_mode" = "initial" + "include_tables" = "${table1},${table2}", + "offset" = "initial" ) TO DATABASE ${currentDb} ( "table.create.properties.replication_num" = "1" ) """ // check table created - def showTables = sql """ show tables from ${currentDb} like '${tableName}'; """ + def showTables = sql """ show tables from ${currentDb} like '${table1}'; """ assert showTables.size() == 1 + def showTables2 = sql """ show tables from ${currentDb} like '${table2}'; """ + assert showTables2.size() == 1 // check job running try { @@ -94,20 +103,27 @@ suite("test_streaming_mysql_job", "p0,external,mysql,external_docker,external_do } // check snapshot data - qt_select """ SELECT * FROM ${tableName} order by name asc """ + qt_select """ SELECT * FROM ${table1} order by name asc """ + qt_select """ SELECT * FROM ${table2} order by name asc """ // mock mysql incremental into connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { - sql """INSERT INTO ${mysqlDb}.${tableName} (name,age) VALUES ('Doris',18);""" - sql """UPDATE ${mysqlDb}.${tableName} SET age = 10 WHERE name = 'B';""" - sql """DELETE FROM ${mysqlDb}.${tableName} WHERE name = 'C';""" + sql """INSERT INTO ${mysqlDb}.${table1} (name,age) VALUES ('Doris',18);""" + sql """UPDATE ${mysqlDb}.${table1} SET age = 10 WHERE name = 'B1';""" + sql """DELETE FROM ${mysqlDb}.${table1} WHERE name = 'A1';""" } sleep(10000); // wait for cdc incremental data // check incremental data - qt_select """ SELECT * FROM ${tableName} order by name asc """ + qt_select """ SELECT * FROM ${table1} order by name asc """ + + def jobInfo = sql """ + select loadStatistic from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfo: " + jobInfo) + assert jobInfo.get(0).get(0) == "{\"scannedRows\":7,\"loadBytes\":334,\"fileNumber\":0,\"fileSize\":0}" sql """ DROP JOB IF EXISTS where jobname = '${jobName}' From 0a27f8a4d8b8f1b9e65e9c2ec91cf8d69e60920b Mon Sep 17 00:00:00 2001 From: wudi Date: Thu, 11 Dec 2025 12:14:50 +0800 Subject: [PATCH 19/27] fix case --- .../insert/streaming/StreamingInsertJob.java | 9 +-- .../apache/doris/cdcclient/common/Env.java | 5 +- .../service/PipelineCoordinator.java | 23 +++--- .../reader/mysql/MySqlSourceReader.java | 74 +++++++++---------- .../cdc/test_streaming_mysql_job.out | 2 + .../cdc/test_streaming_mysql_job.groovy | 6 +- 6 files changed, 59 insertions(+), 60 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index e59a9204d8ef2e..6fa08f1f553065 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -75,7 +75,6 @@ import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnStateChangeCallback; - import com.google.common.base.Preconditions; import com.google.gson.annotations.SerializedName; import lombok.Getter; @@ -83,7 +82,6 @@ import lombok.extern.log4j.Log4j2; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; - import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; @@ -493,8 +491,10 @@ public List queryAllStreamTasks() { protected void fetchMeta() { try { - if (originTvfProps == null && StringUtils.isNotEmpty(getExecuteSql())) { - this.originTvfProps = getCurrentTvf().getProperties().getMap(); + if (tvfType != null) { + if (originTvfProps == null) { + this.originTvfProps = getCurrentTvf().getProperties().getMap(); + } offsetProvider.fetchRemoteMeta(originTvfProps); } else { offsetProvider.fetchRemoteMeta(new HashMap<>()); @@ -505,7 +505,6 @@ protected void fetchMeta() { "Failed to fetch meta, " + ex.getMessage()); } } - public boolean needScheduleTask() { readLock(); try { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java index aabdaf1df4d1b4..ec2f5710b088d2 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java @@ -21,14 +21,12 @@ import org.apache.doris.cdcclient.source.factory.DataSource; import org.apache.doris.cdcclient.source.factory.SourceReaderFactory; import org.apache.doris.cdcclient.source.reader.SourceReader; - +import lombok.Setter; import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; -import lombok.Setter; - public class Env { private static volatile Env INSTANCE; private final Map jobContexts; @@ -40,7 +38,6 @@ private Env() { public String getBackendHostPort() { return "127.0.0.1:" + backendHttpPort; - // return "10.16.10.6:28949"; } public static Env getCurrentEnv() { diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 5bd8d186bec56b..031abcd4310d26 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -41,7 +41,6 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import io.debezium.data.Envelope; import org.slf4j.Logger; @@ -53,8 +52,6 @@ public class PipelineCoordinator { private static final Logger LOG = LoggerFactory.getLogger(PipelineCoordinator.class); private static final String SPLIT_ID = "splitId"; - private static final String PURE_BINLOG_PHASE = "pureBinlogPhase"; - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); // jobId private final Map batchStreamLoadMap = new ConcurrentHashMap<>(); private final SourceRecordDeserializer> serializer; @@ -113,8 +110,12 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordReq); DorisBatchStreamLoad batchStreamLoad = null; Map metaResponse = new HashMap<>(); + boolean hasData = false; + long scannedRows = 0L; + long scannedBytes = 0L; + SplitReadResult readResult = null; try { - SplitReadResult readResult = sourceReader.readSplitRecords(writeRecordReq); + readResult = sourceReader.readSplitRecords(writeRecordReq); batchStreamLoad = getOrCreateBatchStreamLoad( writeRecordReq.getJobId(), writeRecordReq.getTargetDb()); @@ -122,13 +123,10 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { batchStreamLoad.setFrontendAddress(writeRecordReq.getFrontendAddress()); batchStreamLoad.setToken(writeRecordReq.getToken()); - boolean hasData = false; // Record start time for maxInterval check long startTime = System.currentTimeMillis(); long maxIntervalMillis = writeRecordReq.getMaxInterval() * 1000; - long scannedRows = 0L; - long scannedBytes = 0L; // Use iterators to read and write. Iterator iterator = readResult.getRecordIterator(); while (iterator != null && iterator.hasNext()) { @@ -167,6 +165,11 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { break; } } + } finally { + sourceReader.finishSplitRecords(); + } + + try { if (!hasData) { // todo: need return the lastest heartbeat offset, means the maximum offset that the // current job can recover. @@ -196,11 +199,9 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { } // request fe api batchStreamLoad.commitOffset(metaResponse, scannedRows, scannedBytes); + } finally { - sourceReader.finishSplitRecords(); - if (batchStreamLoad != null) { - batchStreamLoad.resetTaskId(); - } + batchStreamLoad.resetTaskId(); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index cfef866a1bcc1f..12522244d9844e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -33,7 +33,18 @@ import org.apache.doris.cdcclient.source.split.BinlogSplit; import org.apache.doris.cdcclient.source.split.SnapshotSplit; import org.apache.doris.cdcclient.utils.ConfigUtil; - +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlPartition; +import io.debezium.document.Array; +import io.debezium.relational.Column; +import io.debezium.relational.TableId; +import io.debezium.relational.history.HistoryRecord; +import io.debezium.relational.history.TableChanges; +import lombok.Getter; +import lombok.Setter; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.apache.flink.api.java.tuple.Tuple2; @@ -43,6 +54,7 @@ import org.apache.flink.cdc.connectors.mysql.debezium.reader.DebeziumReader; import org.apache.flink.cdc.connectors.mysql.debezium.reader.SnapshotSplitReader; import org.apache.flink.cdc.connectors.mysql.debezium.task.context.StatefulTaskContext; +import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; import org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlSnapshotSplitAssigner; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; @@ -61,7 +73,8 @@ import org.apache.flink.cdc.debezium.history.FlinkJsonTableChangeSerializer; import org.apache.flink.table.types.logical.RowType; import org.apache.kafka.connect.source.SourceRecord; - +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.sql.SQLException; import java.util.ArrayList; @@ -74,28 +87,10 @@ import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; -import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.github.shyiko.mysql.binlog.BinaryLogClient; -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.connector.mysql.MySqlPartition; -import io.debezium.document.Array; -import io.debezium.relational.Column; -import io.debezium.relational.TableId; -import io.debezium.relational.history.HistoryRecord; -import io.debezium.relational.history.TableChanges; -import lombok.Getter; -import lombok.Setter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - public class MySqlSourceReader implements SourceReader { private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceReader.class); private static ObjectMapper objectMapper = new ObjectMapper(); private static final String SPLIT_ID = "splitId"; - private static final String PURE_BINLOG_PHASE = "pureBinlogPhase"; private static final FlinkJsonTableChangeSerializer TABLE_CHANGE_SERIALIZER = new FlinkJsonTableChangeSerializer(); private SourceRecordDeserializer> serializer; @@ -229,28 +224,31 @@ private RecordWithMeta buildRecordResponse( MySqlSplit split = readResult.getSplit(); MySqlSplitState currentSplitState = readResult.getSplitState(); int count = 0; - // Serialize records and add them to the response (collect from iterator) - Iterator iterator = readResult.getRecordIterator(); - while (iterator != null && iterator.hasNext()) { - SourceRecord element = iterator.next(); - List serializedRecords = - serializer.deserialize(fetchRecord.getConfig(), element); - if (!CollectionUtils.isEmpty(serializedRecords)) { - recordResponse.getRecords().addAll(serializedRecords); - count += serializedRecords.size(); - // update meta - Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); - if (split.isBinlogSplit()) { - lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); - recordResponse.setMeta(lastMeta); - } - if (count >= fetchRecord.getFetchSize()) { - return recordResponse; + try { + // Serialize records and add them to the response (collect from iterator) + Iterator iterator = readResult.getRecordIterator(); + while (iterator != null && iterator.hasNext()) { + SourceRecord element = iterator.next(); + List serializedRecords = + serializer.deserialize(fetchRecord.getConfig(), element); + if (!CollectionUtils.isEmpty(serializedRecords)) { + recordResponse.getRecords().addAll(serializedRecords); + count += serializedRecords.size(); + // update meta + Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); + if (split.isBinlogSplit()) { + lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); + recordResponse.setMeta(lastMeta); + } + if (count >= fetchRecord.getFetchSize()) { + return recordResponse; + } } } + } finally { + finishSplitRecords(); } - finishSplitRecords(); // Set meta information if (split.isSnapshotSplit() && currentSplitState != null) { BinlogOffset highWatermark = diff --git a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out index 4bb6ad2eb2fe4f..6d72f328d57198 100644 --- a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out +++ b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out @@ -4,6 +4,8 @@ A1 1 B1 2 -- !select -- +A2 1 +B2 2 -- !select -- B1 10 diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy index 41bca762a8e679..a5a4d47c9f5a6f 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy @@ -90,8 +90,8 @@ suite("test_streaming_mysql_job", "p0,external,mysql,external_docker,external_do { def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ log.info("jobSuccendCount: " + jobSuccendCount) - // check job status and succeed task count larger than 1 - jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + // check job status and succeed task count larger than 2 + jobSuccendCount.size() == 1 && '2' <= jobSuccendCount.get(0).get(0) } ) } catch (Exception ex){ @@ -102,8 +102,10 @@ suite("test_streaming_mysql_job", "p0,external,mysql,external_docker,external_do throw ex; } + // check snapshot data qt_select """ SELECT * FROM ${table1} order by name asc """ + qt_select """ SELECT * FROM ${table2} order by name asc """ // mock mysql incremental into From 774821161287c94e90cce7abcd0086f096b2d47c Mon Sep 17 00:00:00 2001 From: wudi Date: Thu, 11 Dec 2025 12:20:29 +0800 Subject: [PATCH 20/27] fix --- .../job/extensions/insert/streaming/StreamingInsertJob.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 6fa08f1f553065..dd86114b0a16b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -75,6 +75,7 @@ import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnStateChangeCallback; + import com.google.common.base.Preconditions; import com.google.gson.annotations.SerializedName; import lombok.Getter; @@ -82,6 +83,7 @@ import lombok.extern.log4j.Log4j2; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; + import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; @@ -505,6 +507,7 @@ protected void fetchMeta() { "Failed to fetch meta, " + ex.getMessage()); } } + public boolean needScheduleTask() { readLock(); try { From b26a7b2ab3635f19d0943dcf0c42f4b18657ac58 Mon Sep 17 00:00:00 2001 From: wudi Date: Thu, 11 Dec 2025 12:27:12 +0800 Subject: [PATCH 21/27] fix --- .../job/extensions/insert/streaming/StreamingInsertJob.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index dd86114b0a16b9..164708d7e51d66 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -507,7 +507,7 @@ protected void fetchMeta() { "Failed to fetch meta, " + ex.getMessage()); } } - + public boolean needScheduleTask() { readLock(); try { From 91c3bd2f7d917cbf726f666dec4627b38b2c30bc Mon Sep 17 00:00:00 2001 From: wudi Date: Thu, 11 Dec 2025 16:00:58 +0800 Subject: [PATCH 22/27] fix --- .../doris/job/cdc}/AbstractSourceSplit.java | 4 +- .../apache/doris/job/cdc}/BinlogSplit.java | 2 +- .../doris/job/cdc/DataSourceConfigKeys.java} | 6 +- .../apache/doris/job/cdc}/SnapshotSplit.java | 2 +- .../insert/streaming/StreamingInsertJob.java | 22 +- .../streaming/StreamingMultiTblTask.java | 4 +- .../doris/job/offset/jdbc/JdbcOffset.java | 4 +- .../offset/jdbc/JdbcSourceOffsetProvider.java | 12 +- .../job/offset/jdbc/split/SourceSplit.java | 20 -- .../doris/job/util/StreamingJobUtils.java | 25 ++- fs_brokers/cdc_client/pom.xml | 31 +-- .../Constants.java} | 6 +- .../apache/doris/cdcclient/common/Env.java | 12 +- .../config/GlobalExceptionHandler.java | 8 +- .../cdcclient/constants/LoadConstants.java | 37 ---- .../controller/ClientController.java | 32 ++- .../cdcclient/model/rest/ResponseBody.java | 134 ----------- .../model/rest/ResponseEntityBuilder.java | 81 ------- .../model/rest/RestApiStatusCode.java | 33 --- .../cdcclient/model/rest/RestResponse.java | 54 +++++ .../service/PipelineCoordinator.java | 50 ++--- .../doris/cdcclient/sink/HttpPutBuilder.java | 4 +- .../deserialize/DebeziumJsonDeserializer.java | 6 +- .../source/factory/SourceReaderFactory.java | 8 +- .../cdcclient/source/reader/SourceReader.java | 47 ++-- .../source/reader/SplitReadResult.java | 16 +- .../reader/mysql/MySqlSourceReader.java | 209 ++++++++---------- .../source/split/AbstractSourceSplit.java | 38 ---- .../cdcclient/source/split/BinlogSplit.java | 46 ---- .../cdcclient/source/split/SnapshotSplit.java | 56 ----- .../doris/cdcclient/utils/ConfigUtil.java | 30 +-- 31 files changed, 299 insertions(+), 740 deletions(-) rename fe/{fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split => fe-common/src/main/java/org/apache/doris/job/cdc}/AbstractSourceSplit.java (90%) rename fe/{fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split => fe-common/src/main/java/org/apache/doris/job/cdc}/BinlogSplit.java (97%) rename fe/{fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java => fe-common/src/main/java/org/apache/doris/job/cdc/DataSourceConfigKeys.java} (91%) rename fe/{fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split => fe-common/src/main/java/org/apache/doris/job/cdc}/SnapshotSplit.java (97%) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SourceSplit.java rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/{source/split/SourceSplit.java => common/Constants.java} (84%) delete mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java delete mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseBody.java delete mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseEntityBuilder.java delete mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestApiStatusCode.java create mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestResponse.java delete mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java delete mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java delete mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/AbstractSourceSplit.java similarity index 90% rename from fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/AbstractSourceSplit.java index a4cf3a01285786..c7cb3c85f905c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/AbstractSourceSplit.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/AbstractSourceSplit.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.job.offset.jdbc.split; +package org.apache.doris.job.cdc; import lombok.AllArgsConstructor; import lombok.Getter; @@ -28,7 +28,7 @@ @Setter @AllArgsConstructor @NoArgsConstructor -public abstract class AbstractSourceSplit implements SourceSplit, Serializable { +public abstract class AbstractSourceSplit implements Serializable { private static final long serialVersionUID = 1L; protected String splitId; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/BinlogSplit.java similarity index 97% rename from fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/BinlogSplit.java index 1494e47416f7c2..6dc7fc35247762 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/BinlogSplit.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/BinlogSplit.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.job.offset.jdbc.split; +package org.apache.doris.job.cdc; import com.google.gson.Gson; import lombok.EqualsAndHashCode; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/DataSourceConfigKeys.java similarity index 91% rename from fe/fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/DataSourceConfigKeys.java index 8f017462c97a18..074c100579c154 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/common/LoadConstants.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/DataSourceConfigKeys.java @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.job.common; +package org.apache.doris.job.cdc; -public class LoadConstants { +public class DataSourceConfigKeys { public static final String JDBC_URL = "jdbc_url"; public static final String DRIVER_URL = "driver_url"; public static final String DRIVER_CLASS = "driver_class"; @@ -32,6 +32,4 @@ public class LoadConstants { public static final String OFFSET_EARLIEST = "earliest"; public static final String OFFSET_LATEST = "latest"; public static final String SPLIT_SIZE = "split_size"; - public static final String TABLE_PROPS_PREFIX = "table.create.properties."; - } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/SnapshotSplit.java similarity index 97% rename from fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/SnapshotSplit.java index eb5f992c345535..44977584d79204 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SnapshotSplit.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/SnapshotSplit.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.job.offset.jdbc.split; +package org.apache.doris.job.cdc; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 164708d7e51d66..835aa0240483ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -37,12 +37,12 @@ import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.base.TimerDefinition; +import org.apache.doris.job.cdc.DataSourceConfigKeys; import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.common.FailureReason; import org.apache.doris.job.common.IntervalUnit; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.JobType; -import org.apache.doris.job.common.LoadConstants; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; @@ -219,23 +219,23 @@ private void initSourceJob() { } private void checkRequiredSourceProperties() { - Preconditions.checkArgument(sourceProperties.get(LoadConstants.JDBC_URL) != null, + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.JDBC_URL) != null, "jdbc_url is required property"); - Preconditions.checkArgument(sourceProperties.get(LoadConstants.DRIVER_URL) != null, + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.DRIVER_URL) != null, "driver_url is required property"); - Preconditions.checkArgument(sourceProperties.get(LoadConstants.DRIVER_CLASS) != null, + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.DRIVER_CLASS) != null, "driver_class is required property"); - Preconditions.checkArgument(sourceProperties.get(LoadConstants.USER) != null, + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.USER) != null, "user is required property"); - Preconditions.checkArgument(sourceProperties.get(LoadConstants.PASSWORD) != null, + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.PASSWORD) != null, "password is required property"); - Preconditions.checkArgument(sourceProperties.get(LoadConstants.DATABASE) != null, + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.DATABASE) != null, "database is required property"); - Preconditions.checkArgument(sourceProperties.get(LoadConstants.INCLUDE_TABLES) != null - || sourceProperties.get(LoadConstants.EXCLUDE_TABLES) != null, + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.INCLUDE_TABLES) != null + || sourceProperties.get(DataSourceConfigKeys.EXCLUDE_TABLES) != null, "Either include_tables or exclude_tables must be specified"); - if (!sourceProperties.containsKey(LoadConstants.OFFSET)) { - sourceProperties.put(LoadConstants.OFFSET, LoadConstants.OFFSET_LATEST); + if (!sourceProperties.containsKey(DataSourceConfigKeys.OFFSET)) { + sourceProperties.put(DataSourceConfigKeys.OFFSET, DataSourceConfigKeys.OFFSET_LATEST); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java index 3f73ba47bd3381..4a711764f569e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java @@ -24,14 +24,14 @@ import org.apache.doris.httpv2.rest.RestApiStatusCode; import org.apache.doris.httpv2.rest.StreamingJobAction.CommitOffsetRequest; import org.apache.doris.job.base.Job; +import org.apache.doris.job.cdc.BinlogSplit; +import org.apache.doris.job.cdc.SnapshotSplit; import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.offset.SourceOffsetProvider; import org.apache.doris.job.offset.jdbc.JdbcOffset; import org.apache.doris.job.offset.jdbc.JdbcSourceOffsetProvider; -import org.apache.doris.job.offset.jdbc.split.BinlogSplit; -import org.apache.doris.job.offset.jdbc.split.SnapshotSplit; import org.apache.doris.job.util.StreamingJobUtils; import org.apache.doris.proto.InternalService; import org.apache.doris.proto.InternalService.PRequestCdcClientResult; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java index e3eb37c716c8c0..beba9f5d7d2b6f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -17,9 +17,9 @@ package org.apache.doris.job.offset.jdbc; +import org.apache.doris.job.cdc.AbstractSourceSplit; +import org.apache.doris.job.cdc.BinlogSplit; import org.apache.doris.job.offset.Offset; -import org.apache.doris.job.offset.jdbc.split.AbstractSourceSplit; -import org.apache.doris.job.offset.jdbc.split.BinlogSplit; import com.google.gson.Gson; import lombok.AllArgsConstructor; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index 45075f3c3f2871..d5b29e581d4e06 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -18,16 +18,16 @@ package org.apache.doris.job.offset.jdbc; import org.apache.doris.httpv2.entity.ResponseBody; +import org.apache.doris.job.cdc.AbstractSourceSplit; +import org.apache.doris.job.cdc.BinlogSplit; +import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.cdc.SnapshotSplit; import org.apache.doris.job.common.DataSourceType; -import org.apache.doris.job.common.LoadConstants; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; -import org.apache.doris.job.offset.jdbc.split.AbstractSourceSplit; -import org.apache.doris.job.offset.jdbc.split.BinlogSplit; -import org.apache.doris.job.offset.jdbc.split.SnapshotSplit; import org.apache.doris.job.util.StreamingJobUtils; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.persist.gson.GsonUtils; @@ -506,10 +506,10 @@ private Map buildSplitParams(String table) { } private boolean checkNeedSplitChunks(Map sourceProperties) { - String startMode = sourceProperties.get(LoadConstants.OFFSET); + String startMode = sourceProperties.get(DataSourceConfigKeys.OFFSET); if (startMode == null) { return false; } - return LoadConstants.OFFSET_INITIAL.equalsIgnoreCase(startMode); + return DataSourceConfigKeys.OFFSET_INITIAL.equalsIgnoreCase(startMode); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SourceSplit.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SourceSplit.java deleted file mode 100644 index 5bf6f2c1b2bf70..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/split/SourceSplit.java +++ /dev/null @@ -1,20 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.job.offset.jdbc.split; - -public interface SourceSplit {} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java index db237598305344..2004073fe18ae8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java @@ -28,10 +28,10 @@ import org.apache.doris.datasource.jdbc.client.JdbcClient; import org.apache.doris.datasource.jdbc.client.JdbcClientConfig; import org.apache.doris.datasource.jdbc.client.JdbcMySQLClient; +import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.cdc.SnapshotSplit; import org.apache.doris.job.common.DataSourceType; -import org.apache.doris.job.common.LoadConstants; import org.apache.doris.job.exception.JobException; -import org.apache.doris.job.offset.jdbc.split.SnapshotSplit; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; @@ -66,6 +66,7 @@ @Log4j2 public class StreamingJobUtils { + public static final String TABLE_PROPS_PREFIX = "table.create.properties."; public static final String INTERNAL_STREAMING_JOB_META_TABLE_NAME = "streaming_job_meta"; public static final String FULL_QUALIFIED_META_TBL_NAME = InternalCatalog.INTERNAL_CATALOG_NAME + "." + FeConstants.INTERNAL_DB_NAME + "." + INTERNAL_STREAMING_JOB_META_TABLE_NAME; @@ -190,11 +191,11 @@ private static JdbcClient getJdbcClient(DataSourceType sourceType, Map generateCreateTableCmds(String targetDb, Map properties, Map targetProperties) throws JobException { List createtblCmds = new ArrayList<>(); - String includeTables = properties.get(LoadConstants.INCLUDE_TABLES); - String excludeTables = properties.get(LoadConstants.EXCLUDE_TABLES); + String includeTables = properties.get(DataSourceConfigKeys.INCLUDE_TABLES); + String excludeTables = properties.get(DataSourceConfigKeys.EXCLUDE_TABLES); List includeTablesList = new ArrayList<>(); if (includeTables != null) { includeTablesList = Arrays.asList(includeTables.split(",")); } - String database = properties.get(LoadConstants.DATABASE); + String database = properties.get(DataSourceConfigKeys.DATABASE); JdbcClient jdbcClient = getJdbcClient(sourceType, properties); List tablesNameList = jdbcClient.getTablesNameList(database); if (tablesNameList.isEmpty()) { @@ -307,8 +308,8 @@ public static List generateCreateTableCmds(String targetDb, private static Map getTableCreateProperties(Map properties) { final Map tableCreateProps = new HashMap<>(); for (Map.Entry entry : properties.entrySet()) { - if (entry.getKey().startsWith(LoadConstants.TABLE_PROPS_PREFIX)) { - String subKey = entry.getKey().substring(LoadConstants.TABLE_PROPS_PREFIX.length()); + if (entry.getKey().startsWith(TABLE_PROPS_PREFIX)) { + String subKey = entry.getKey().substring(TABLE_PROPS_PREFIX.length()); tableCreateProps.put(subKey, entry.getValue()); } } diff --git a/fs_brokers/cdc_client/pom.xml b/fs_brokers/cdc_client/pom.xml index 3e0c1dbb46e888..1f5723f9b9cacf 100644 --- a/fs_brokers/cdc_client/pom.xml +++ b/fs_brokers/cdc_client/pom.xml @@ -74,6 +74,17 @@ under the License. + + org.apache.doris + fe-common + ${doris.version} + + + * + * + + + org.springframework.boot spring-boot-starter-web @@ -87,26 +98,6 @@ under the License. org.springframework.boot spring-boot-starter-logging - - - - - - - - - - - - - - - - - - - - diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SourceSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java similarity index 84% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SourceSplit.java rename to fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java index 274927d43e4c61..f7a49bdaa08c01 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SourceSplit.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.source.split; +package org.apache.doris.cdcclient.common; -public interface SourceSplit {} +public class Constants { + public static final String DORIS_DELETE_SIGN = "__DORIS_DELETE_SIGN__"; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java index ec2f5710b088d2..be8bf466abe8f2 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java @@ -21,12 +21,14 @@ import org.apache.doris.cdcclient.source.factory.DataSource; import org.apache.doris.cdcclient.source.factory.SourceReaderFactory; import org.apache.doris.cdcclient.source.reader.SourceReader; -import lombok.Setter; + import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; +import lombok.Setter; + public class Env { private static volatile Env INSTANCE; private final Map jobContexts; @@ -51,7 +53,7 @@ public static Env getCurrentEnv() { return INSTANCE; } - public SourceReader getReader(JobConfig jobConfig) { + public SourceReader getReader(JobConfig jobConfig) { DataSource ds = resolveDataSource(jobConfig.getDataSource()); Env manager = Env.getCurrentEnv(); return manager.getOrCreateReader(jobConfig.getJobId(), ds, jobConfig.getConfig()); @@ -68,7 +70,7 @@ private DataSource resolveDataSource(String source) { } } - private SourceReader getOrCreateReader( + private SourceReader getOrCreateReader( Long jobId, DataSource dataSource, Map config) { JobContext context = getOrCreateContext(jobId, dataSource, config); return context.getOrCreateReader(dataSource); @@ -90,7 +92,7 @@ private JobContext getOrCreateContext( private static final class JobContext { private final long jobId; - private volatile SourceReader reader; + private volatile SourceReader reader; private volatile Map config; private volatile DataSource dataSource; @@ -100,7 +102,7 @@ private JobContext(long jobId, DataSource dataSource, Map config this.config = config; } - private synchronized SourceReader getOrCreateReader(DataSource source) { + private synchronized SourceReader getOrCreateReader(DataSource source) { if (reader == null) { reader = SourceReaderFactory.createSourceReader(source); reader.initialize(); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java index 756e323df462a7..8b4883b6203183 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java @@ -17,12 +17,10 @@ package org.apache.doris.cdcclient.config; -import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; +import org.apache.doris.cdcclient.model.rest.RestResponse; import jakarta.servlet.http.HttpServletRequest; import lombok.extern.slf4j.Slf4j; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.context.MessageSource; import org.springframework.web.bind.annotation.ControllerAdvice; import org.springframework.web.bind.annotation.ExceptionHandler; import org.springframework.web.bind.annotation.ResponseBody; @@ -31,12 +29,10 @@ @ControllerAdvice public class GlobalExceptionHandler { - @Autowired private MessageSource messageSource; - @ExceptionHandler(Exception.class) @ResponseBody public Object exceptionHandler(HttpServletRequest request, Exception e) { log.error("Unexpected exception", e); - return ResponseEntityBuilder.internalError(e.getMessage()); + return RestResponse.internalError(e.getMessage()); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java deleted file mode 100644 index bfc1d27bc18fc7..00000000000000 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/constants/LoadConstants.java +++ /dev/null @@ -1,37 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.cdcclient.constants; - -public class LoadConstants { - public static final String JDBC_URL = "jdbc_url"; - public static final String DRIVER_URL = "driver_url"; - public static final String DRIVER_CLASS = "driver_class"; - public static final String USER = "user"; - public static final String PASSWORD = "password"; - public static final String DATABASE = "database"; - public static final String INCLUDE_TABLES = "include_tables"; - public static final String EXCLUDE_TABLES = "exclude_tables"; - // initial,earliest,latest,{binlog,postion},\d{13} - public static final String OFFSET = "offset"; - public static final String OFFSET_INITIAL = "initial"; - public static final String OFFSET_EARLIEST = "earliest"; - public static final String OFFSET_LATEST = "latest"; - public static final String SPLIT_SIZE = "split_size"; - public static final String TABLE_PROPS_PREFIX = "table.create.properties."; - public static final String DELETE_SIGN_KEY = "__DORIS_DELETE_SIGN__"; -} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index f3fddad95c3a73..ca3fd2e5c61870 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -23,13 +23,12 @@ import org.apache.doris.cdcclient.model.request.FetchRecordReq; import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; import org.apache.doris.cdcclient.model.request.WriteRecordReq; -import org.apache.doris.cdcclient.model.rest.ResponseEntityBuilder; +import org.apache.doris.cdcclient.model.rest.RestResponse; import org.apache.doris.cdcclient.service.PipelineCoordinator; import org.apache.doris.cdcclient.source.reader.SourceReader; import java.util.List; -import jakarta.servlet.http.HttpServletRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -49,12 +48,12 @@ public class ClientController { @RequestMapping(path = "/api/fetchSplits", method = RequestMethod.POST) public Object fetchSplits(@RequestBody FetchTableSplitsReq ftsReq) { try { - SourceReader reader = Env.getCurrentEnv().getReader(ftsReq); + SourceReader reader = Env.getCurrentEnv().getReader(ftsReq); List splits = reader.getSourceSplits(ftsReq); - return ResponseEntityBuilder.ok(splits); + return RestResponse.success(splits); } catch (IllegalArgumentException ex) { LOG.error("Failed to fetch splits, jobId={}", ftsReq.getJobId(), ex); - return ResponseEntityBuilder.badRequest(ex.getMessage()); + return RestResponse.internalError(ex.getMessage()); } } @@ -62,11 +61,11 @@ public Object fetchSplits(@RequestBody FetchTableSplitsReq ftsReq) { @RequestMapping(path = "/api/fetchRecords", method = RequestMethod.POST) public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { try { - SourceReader reader = Env.getCurrentEnv().getReader(recordReq); - return ResponseEntityBuilder.ok(reader.read(recordReq)); + SourceReader reader = Env.getCurrentEnv().getReader(recordReq); + return RestResponse.success(reader.read(recordReq)); } catch (Exception ex) { LOG.error("Failed fetch record, jobId={}", recordReq.getJobId(), ex); - return ResponseEntityBuilder.badRequest(ex.getMessage()); + return RestResponse.internalError(ex.getMessage()); } } @@ -79,32 +78,29 @@ public Object writeRecord(@RequestBody WriteRecordReq recordReq) { recordReq.getTaskId(), recordReq.getMeta()); pipelineCoordinator.writeRecordsAsync(recordReq); - return ResponseEntityBuilder.ok("Request accepted, processing asynchronously"); - } - - private String parseToken(HttpServletRequest request) { - return request.getHeader("token"); + return RestResponse.success("Request accepted, processing asynchronously"); } /** Fetch lastest end meta */ @RequestMapping(path = "/api/fetchEndOffset", method = RequestMethod.POST) public Object fetchEndOffset(@RequestBody JobConfig jobConfig) { - SourceReader reader = Env.getCurrentEnv().getReader(jobConfig); - return ResponseEntityBuilder.ok(reader.getEndOffset(jobConfig)); + SourceReader reader = Env.getCurrentEnv().getReader(jobConfig); + return RestResponse.success(reader.getEndOffset(jobConfig)); } /** compare datasource Binlog Offset */ @RequestMapping(path = "/api/compareOffset", method = RequestMethod.POST) public Object compareOffset(@RequestBody CompareOffsetReq compareOffsetReq) { - SourceReader reader = Env.getCurrentEnv().getReader(compareOffsetReq); - return ResponseEntityBuilder.ok(reader.compareOffset(compareOffsetReq)); + SourceReader reader = Env.getCurrentEnv().getReader(compareOffsetReq); + return RestResponse.success(reader.compareOffset(compareOffsetReq)); } + /** Close job */ @RequestMapping(path = "/api/close/{jobId}", method = RequestMethod.POST) public Object close(@PathVariable long jobId) { Env env = Env.getCurrentEnv(); env.close(jobId); pipelineCoordinator.closeJob(jobId); - return ResponseEntityBuilder.ok(true); + return RestResponse.success(true); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseBody.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseBody.java deleted file mode 100644 index 96b37050b5fbb8..00000000000000 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseBody.java +++ /dev/null @@ -1,134 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.cdcclient.model.rest; - -import java.util.Objects; - -/** - * The response body of restful api. - * - *

The getter setter methods of all member variables need to be retained to ensure that Spring - * can perform json format conversion. - * - * @param type of data - */ -public class ResponseBody { - // Used to describe the error message. If there are no errors, it displays "OK" - private String msg; - // The user displays an error code. - // If there is no error, 0 is displayed. - // If there is an error, it is usually Doris's internal error code, not the HTTP standard error - // code. - // The HTTP standard error code should be reflected in the return value of the HTTP protocol. - private int code = RestApiStatusCode.OK.code; - // to save the response body - private T data; - // to save the number of records in response body. - // currently not used and always be 0. - private int count; - - public ResponseBody() {} - - public ResponseBody msg(String msg) { - this.msg = msg; - return this; - } - - public ResponseBody code(RestApiStatusCode code) { - this.code = code.code; - return this; - } - - public ResponseBody data(T data) { - this.data = data; - return this; - } - - public String getMsg() { - return msg; - } - - public void setMsg(String msg) { - this.msg = msg; - } - - public int getCode() { - return code; - } - - public void setCode(int code) { - this.code = code; - } - - public T getData() { - return data; - } - - public void setData(T data) { - this.data = data; - } - - public void setCount(int count) { - this.count = count; - } - - public int getCount() { - return count; - } - - public ResponseBody commonError(String msg) { - this.code = RestApiStatusCode.COMMON_ERROR.code; - this.msg = msg; - return this; - } - - @Override - public String toString() { - return "ResponseBody{" - + "msg='" - + msg - + '\'' - + ", code=" - + code - + ", data=" - + data - + ", count=" - + count - + '}'; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ResponseBody that = (ResponseBody) o; - return code == that.code - && count == that.count - && Objects.equals(msg, that.msg) - && Objects.equals(data, that.data); - } - - @Override - public int hashCode() { - return Objects.hash(msg, code, data, count); - } -} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseEntityBuilder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseEntityBuilder.java deleted file mode 100644 index 38b5dd2aa0aece..00000000000000 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/ResponseEntityBuilder.java +++ /dev/null @@ -1,81 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.cdcclient.model.rest; - -import org.springframework.http.HttpStatus; -import org.springframework.http.ResponseEntity; - -/** - * A utility class for creating a ResponseEntity easier. All response will return with http code - * 200, and a internal code represent the real code. - */ -public class ResponseEntityBuilder { - - public static ResponseEntity badRequest(Object data) { - ResponseBody body = - new ResponseBody() - .code(RestApiStatusCode.BAD_REQUEST) - .msg("Bad Request") - .data(data); - return ResponseEntity.status(HttpStatus.OK).body(body); - } - - public static ResponseEntity okWithCommonError(String msg) { - ResponseBody body = - new ResponseBody().code(RestApiStatusCode.COMMON_ERROR).msg("Error").data(msg); - return ResponseEntity.status(HttpStatus.OK).body(body); - } - - public static ResponseEntity ok(Object data) { - ResponseBody body = new ResponseBody().code(RestApiStatusCode.OK).msg("success").data(data); - return ResponseEntity.status(HttpStatus.OK).body(body); - } - - public static ResponseEntity ok() { - ResponseBody body = new ResponseBody().code(RestApiStatusCode.OK).msg("success"); - return ResponseEntity.status(HttpStatus.OK).body(body); - } - - public static ResponseEntity okWithEmpty() { - return ResponseEntity.status(HttpStatus.OK).build(); - } - - public static ResponseEntity unauthorized(Object data) { - ResponseBody body = - new ResponseBody() - .code(RestApiStatusCode.UNAUTHORIZED) - .msg("Unauthorized") - .data(data); - return ResponseEntity.status(HttpStatus.OK).body(body); - } - - public static ResponseEntity internalError(Object data) { - ResponseBody body = - new ResponseBody() - .code(RestApiStatusCode.INTERNAL_SERVER_ERROR) - .msg("Internal Error") - .data(data); - return ResponseEntity.status(HttpStatus.OK).body(body); - } - - public static ResponseEntity notFound(Object data) { - ResponseBody body = - new ResponseBody().code(RestApiStatusCode.NOT_FOUND).msg("Not Found").data(data); - return ResponseEntity.status(HttpStatus.OK).body(body); - } -} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestApiStatusCode.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestApiStatusCode.java deleted file mode 100644 index a5ea74587038b7..00000000000000 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestApiStatusCode.java +++ /dev/null @@ -1,33 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.cdcclient.model.rest; - -public enum RestApiStatusCode { - OK(0), - COMMON_ERROR(1), - UNAUTHORIZED(401), - BAD_REQUEST(403), - NOT_FOUND(404), - INTERNAL_SERVER_ERROR(500); - - public int code; - - RestApiStatusCode(int code) { - this.code = code; - } -} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestResponse.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestResponse.java new file mode 100644 index 00000000000000..5126f138c2db48 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestResponse.java @@ -0,0 +1,54 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model.rest; + +import java.io.Serializable; + +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@NoArgsConstructor +public class RestResponse implements Serializable { + private static final long serialVersionUID = 1L; + public static final int SUCCESS = 0; + public static final int FAIL = 1; + + private static final String DEFAULT_SUCCESS_MESSAGE = "Success"; + private static final String DEFAULT_SYSTEM_ERROR_MESSAGE = "Internal Error"; + + private String msg; + private int code; + private T data; + + public static RestResponse success(T data) { + RestResponse response = new RestResponse<>(); + response.setCode(SUCCESS); + response.setMsg(DEFAULT_SUCCESS_MESSAGE); + response.setData(data); + return response; + } + + public static RestResponse internalError(T data) { + RestResponse response = new RestResponse<>(); + response.setCode(FAIL); + response.setMsg(DEFAULT_SYSTEM_ERROR_MESSAGE); + response.setData(data); + return response; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 031abcd4310d26..8f9211a8e4ca43 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -107,13 +107,13 @@ public CompletableFuture writeRecordsAsync(WriteRecordReq writeRecordReq) /** Read data from SourceReader and write it to Doris, while returning meta information. */ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { - SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordReq); + SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordReq); DorisBatchStreamLoad batchStreamLoad = null; Map metaResponse = new HashMap<>(); boolean hasData = false; long scannedRows = 0L; long scannedBytes = 0L; - SplitReadResult readResult = null; + SplitReadResult readResult = null; try { readResult = sourceReader.readSplitRecords(writeRecordReq); batchStreamLoad = @@ -131,30 +131,26 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { Iterator iterator = readResult.getRecordIterator(); while (iterator != null && iterator.hasNext()) { SourceRecord element = iterator.next(); - if (RecordUtils.isDataChangeRecord(element)) { - List serializedRecords = - serializer.deserialize(writeRecordReq.getConfig(), element); - if (!CollectionUtils.isEmpty(serializedRecords)) { - String database = writeRecordReq.getTargetDb(); - String table = extractTable(element); - hasData = true; - for (String record : serializedRecords) { - scannedRows++; - byte[] dataBytes = record.getBytes(); - scannedBytes += dataBytes.length; - batchStreamLoad.writeRecord(database, table, dataBytes); - } - - Map lastMeta = - RecordUtils.getBinlogPosition(element).getOffset(); - if (sourceReader.isBinlogSplit(readResult.getSplit()) - && sourceReader.getSplitId(readResult.getSplit()) != null) { - lastMeta.put(SPLIT_ID, sourceReader.getSplitId(readResult.getSplit())); - } - metaResponse = lastMeta; + List serializedRecords = + serializer.deserialize(writeRecordReq.getConfig(), element); + if (!CollectionUtils.isEmpty(serializedRecords)) { + String database = writeRecordReq.getTargetDb(); + String table = extractTable(element); + hasData = true; + for (String record : serializedRecords) { + scannedRows++; + byte[] dataBytes = record.getBytes(); + scannedBytes += dataBytes.length; + batchStreamLoad.writeRecord(database, table, dataBytes); } - } else { - LOG.info("Skip non-data record: {}", element.valueSchema()); + + Map lastMeta = + RecordUtils.getBinlogPosition(element).getOffset(); + if (sourceReader.isBinlogSplit(readResult.getSplit()) + && readResult.getSplit() != null) { + lastMeta.put(SPLIT_ID, readResult.getSplit().splitId()); + } + metaResponse = lastMeta; } // Check if maxInterval has been exceeded long elapsedTime = System.currentTimeMillis() - startTime; @@ -172,7 +168,7 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { try { if (!hasData) { // todo: need return the lastest heartbeat offset, means the maximum offset that the - // current job can recover. + // current job can recover. if (sourceReader.isBinlogSplit(readResult.getSplit())) { Map offsetRes = sourceReader.extractBinlogOffset(readResult.getSplit()); @@ -189,7 +185,7 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { if (!sourceReader.isBinlogSplit(readResult.getSplit())) { Map offsetRes = sourceReader.extractSnapshotOffset( - readResult.getSplitState(), readResult.getSplit()); + readResult.getSplit(), readResult.getSplitState()); if (offsetRes == null) { // should not happen throw new StreamLoadException( diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java index 7aa177f1f2ef74..c1594e04ffc30e 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java @@ -17,7 +17,7 @@ package org.apache.doris.cdcclient.sink; -import org.apache.doris.cdcclient.constants.LoadConstants; +import org.apache.doris.cdcclient.common.Constants; import org.apache.flink.util.Preconditions; import org.apache.http.HttpEntity; @@ -56,7 +56,7 @@ public HttpPutBuilder addBodyContentType() { public HttpPutBuilder addHiddenColumns(boolean add) { if (add) { - header.put("hidden_columns", LoadConstants.DELETE_SIGN_KEY); + header.put("hidden_columns", Constants.DORIS_DELETE_SIGN); } return this; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java index 67efff55c0a016..9c621518d1013b 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java @@ -37,7 +37,7 @@ import java.util.List; import java.util.Map; -import static org.apache.doris.cdcclient.constants.LoadConstants.DELETE_SIGN_KEY; +import static org.apache.doris.cdcclient.common.Constants.DORIS_DELETE_SIGN; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -112,7 +112,7 @@ private String extractAfterRow(Struct value, Schema valueSchema) convert(field.schema(), after.getWithoutDefault(field.name())); record.put(field.name(), valueConverted); }); - record.put(DELETE_SIGN_KEY, 0); + record.put(DORIS_DELETE_SIGN, 0); return objectMapper.writeValueAsString(record); } @@ -132,7 +132,7 @@ private String extractBeforeRow(Struct value, Schema valueSchema) convert(field.schema(), before.getWithoutDefault(field.name())); record.put(field.name(), valueConverted); }); - record.put(DELETE_SIGN_KEY, 1); + record.put(DORIS_DELETE_SIGN, 1); return objectMapper.writeValueAsString(record); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java index 71aadab377fc75..5a7a91c6f96a56 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java @@ -31,7 +31,7 @@ /** SourceReader register. */ public final class SourceReaderFactory { private static final Logger LOG = LoggerFactory.getLogger(SourceReaderFactory.class); - private static final Map>> REGISTRY = + private static final Map> REGISTRY = new ConcurrentHashMap<>(); static { @@ -40,15 +40,15 @@ public final class SourceReaderFactory { private SourceReaderFactory() {} - public static void register(DataSource source, Supplier> supplier) { + public static void register(DataSource source, Supplier supplier) { Objects.requireNonNull(source, "source"); Objects.requireNonNull(supplier, "supplier"); REGISTRY.put(source, supplier); LOG.info("Registered SourceReader provider for {}", source); } - public static SourceReader createSourceReader(DataSource source) { - Supplier> supplier = REGISTRY.get(source); + public static SourceReader createSourceReader(DataSource source) { + Supplier supplier = REGISTRY.get(source); if (supplier == null) { throw new IllegalArgumentException( "Unsupported SourceReader with datasource : " + source); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index 24863c1e4ace51..4a319aba12bad9 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -23,57 +23,48 @@ import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; import org.apache.doris.cdcclient.model.request.JobBaseRecordReq; import org.apache.doris.cdcclient.model.response.RecordWithMeta; -import org.apache.doris.cdcclient.source.split.SourceSplit; +import org.apache.doris.job.cdc.AbstractSourceSplit; + +import org.apache.flink.api.connector.source.SourceSplit; import java.util.List; import java.util.Map; -/** - * SourceReader interface - * - * @param Split (MySqlSplit) - * @param SplitState(MySqlSplitState) - */ -public interface SourceReader { +/** Source Reader Interface */ +public interface SourceReader { /** Initialization, called when the program starts */ void initialize(); /** Divide the data to be read. For example: split mysql to chunks */ - List getSourceSplits(FetchTableSplitsReq config); + List getSourceSplits(FetchTableSplitsReq config); /** Reading Data */ RecordWithMeta read(FetchRecordReq meta) throws Exception; /** Reading Data for split reader */ - default SplitReadResult readSplitRecords(JobBaseRecordReq baseReq) - throws Exception { - throw new UnsupportedOperationException( - "readSplitRecords is not supported by " + this.getClass().getName()); - } + SplitReadResult readSplitRecords(JobBaseRecordReq baseReq) throws Exception; /** Extract offset information from snapshot split state. */ - Map extractSnapshotOffset(Object splitState, Object split); + Map extractSnapshotOffset(SourceSplit split, Object splitState); /** Extract offset information from binlog split. */ - Map extractBinlogOffset(Object split); - - /** - * Get split ID from the split. This method should be implemented by each SourceReader to handle - * its specific Split type. - * - * @param split the split - * @return split ID, or null if split is null - */ - String getSplitId(Object split); + Map extractBinlogOffset(SourceSplit split); - boolean isBinlogSplit(Object split); + /** Is the split a binlog split */ + boolean isBinlogSplit(SourceSplit split); - /** Called when closing */ - void close(Long jobId); + /** Is the split a snapshot split */ + boolean isSnapshotSplit(SourceSplit split); + /** Finish reading all split records */ void finishSplitRecords(); + /** Get the end offset for the job */ Map getEndOffset(JobConfig jobConfig); + /** Compare the offsets */ int compareOffset(CompareOffsetReq compareOffsetReq); + + /** Called when closing */ + void close(Long jobId); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java index 1b21c3d3818fd5..39f386f89886c1 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java @@ -17,21 +17,19 @@ package org.apache.doris.cdcclient.source.reader; +import org.apache.flink.api.connector.source.SourceSplit; import org.apache.kafka.connect.source.SourceRecord; import java.util.Iterator; import lombok.Data; -/** - * The result of reading a split with iterator. - * - * @param Split type (MySqlSplit) - * @param SplitState type (MySqlSplitState) - */ +/** The result of reading a split with iterator. */ @Data -public class SplitReadResult { +public class SplitReadResult { private Iterator recordIterator; - private SplitState splitState; - private Split split; + // MySqlSplitState, SourceSplitState + private Object splitState; + // MySqlSplit SourceSplitBase + private SourceSplit split; } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index 12522244d9844e..c60024ea2e0131 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -17,7 +17,6 @@ package org.apache.doris.cdcclient.source.reader.mysql; -import org.apache.doris.cdcclient.constants.LoadConstants; import org.apache.doris.cdcclient.model.JobConfig; import org.apache.doris.cdcclient.model.request.CompareOffsetReq; import org.apache.doris.cdcclient.model.request.FetchRecordReq; @@ -29,23 +28,14 @@ import org.apache.doris.cdcclient.source.reader.SourceReader; import org.apache.doris.cdcclient.source.reader.SplitReadResult; import org.apache.doris.cdcclient.source.reader.SplitRecords; -import org.apache.doris.cdcclient.source.split.AbstractSourceSplit; -import org.apache.doris.cdcclient.source.split.BinlogSplit; -import org.apache.doris.cdcclient.source.split.SnapshotSplit; import org.apache.doris.cdcclient.utils.ConfigUtil; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.github.shyiko.mysql.binlog.BinaryLogClient; -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.connector.mysql.MySqlPartition; -import io.debezium.document.Array; -import io.debezium.relational.Column; -import io.debezium.relational.TableId; -import io.debezium.relational.history.HistoryRecord; -import io.debezium.relational.history.TableChanges; -import lombok.Getter; -import lombok.Setter; +import org.apache.doris.job.cdc.AbstractSourceSplit; +import org.apache.doris.job.cdc.BinlogSplit; +import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.cdc.SnapshotSplit; + import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.connector.source.SourceSplit; import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.utils.Preconditions; @@ -54,7 +44,6 @@ import org.apache.flink.cdc.connectors.mysql.debezium.reader.DebeziumReader; import org.apache.flink.cdc.connectors.mysql.debezium.reader.SnapshotSplitReader; import org.apache.flink.cdc.connectors.mysql.debezium.task.context.StatefulTaskContext; -import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; import org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlSnapshotSplitAssigner; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; @@ -73,8 +62,7 @@ import org.apache.flink.cdc.debezium.history.FlinkJsonTableChangeSerializer; import org.apache.flink.table.types.logical.RowType; import org.apache.kafka.connect.source.SourceRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + import java.io.IOException; import java.sql.SQLException; import java.util.ArrayList; @@ -87,18 +75,39 @@ import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; -public class MySqlSourceReader implements SourceReader { +import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlPartition; +import io.debezium.document.Array; +import io.debezium.relational.Column; +import io.debezium.relational.TableId; +import io.debezium.relational.history.HistoryRecord; +import io.debezium.relational.history.TableChanges; +import lombok.Data; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Data +public class MySqlSourceReader implements SourceReader { private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceReader.class); private static ObjectMapper objectMapper = new ObjectMapper(); private static final String SPLIT_ID = "splitId"; private static final FlinkJsonTableChangeSerializer TABLE_CHANGE_SERIALIZER = new FlinkJsonTableChangeSerializer(); private SourceRecordDeserializer> serializer; - private JobRuntimeContext jobRuntimeContext; + private SnapshotSplitReader snapshotReader; + private BinlogSplitReader binlogReader; + private DebeziumReader currentReader; + private Map tableSchemas; + private SplitRecords currentSplitRecords; + private MySqlSplit currentSplit; public MySqlSourceReader() { this.serializer = new DebeziumJsonDeserializer(); - this.jobRuntimeContext = new JobRuntimeContext(); } @Override @@ -152,14 +161,13 @@ public List getSourceSplits(FetchTableSplitsReq ftsReq) { */ @Override public RecordWithMeta read(FetchRecordReq fetchRecord) throws Exception { - SplitReadResult readResult = readSplitRecords(fetchRecord); + SplitReadResult readResult = readSplitRecords(fetchRecord); return buildRecordResponse(fetchRecord, readResult); } /** read split records. */ @Override - public SplitReadResult readSplitRecords(JobBaseRecordReq baseReq) - throws Exception { + public SplitReadResult readSplitRecords(JobBaseRecordReq baseReq) throws Exception { Map offsetMeta = baseReq.getMeta(); if (offsetMeta == null || offsetMeta.isEmpty()) { throw new RuntimeException("miss meta offset"); @@ -169,10 +177,9 @@ public SplitReadResult readSplitRecords(JobBaseReco // If there is an active split being consumed, reuse it directly; // Otherwise, create a new snapshot/binlog split based on offset and start the reader. MySqlSplit split = null; - SplitRecords currentSplitRecords = jobRuntimeContext.getCurrentSplitRecords(); + SplitRecords currentSplitRecords = this.getCurrentSplitRecords(); if (currentSplitRecords == null) { - DebeziumReader currentReader = - jobRuntimeContext.getCurrentReader(); + DebeziumReader currentReader = this.getCurrentReader(); if (currentReader == null || baseReq.isReload()) { LOG.info( "No current reader or reload {}, create new split reader", @@ -181,13 +188,13 @@ public SplitReadResult readSplitRecords(JobBaseReco Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); split = splitFlag.f0; currentSplitRecords = pollSplitRecordsWithSplit(split, baseReq); - jobRuntimeContext.setCurrentSplitRecords(currentSplitRecords); - jobRuntimeContext.setCurrentSplit(split); + this.setCurrentSplitRecords(currentSplitRecords); + this.setCurrentSplit(split); } else if (currentReader instanceof BinlogSplitReader) { LOG.info("Continue poll records with current binlog reader"); // only for binlog reader currentSplitRecords = pollSplitRecordsWithCurrentReader(currentReader); - split = jobRuntimeContext.getCurrentSplit(); + split = this.getCurrentSplit(); } else { throw new RuntimeException("Should not happen"); } @@ -198,9 +205,9 @@ public SplitReadResult readSplitRecords(JobBaseReco } // build response with iterator - SplitReadResult result = new SplitReadResult<>(); + SplitReadResult result = new SplitReadResult(); MySqlSplitState currentSplitState = null; - MySqlSplit currentSplit = jobRuntimeContext.getCurrentSplit(); + MySqlSplit currentSplit = this.getCurrentSplit(); if (currentSplit.isSnapshotSplit()) { currentSplitState = new MySqlSnapshotSplitState(currentSplit.asSnapshotSplit()); } else { @@ -218,11 +225,9 @@ public SplitReadResult readSplitRecords(JobBaseReco /** build RecordWithMeta */ private RecordWithMeta buildRecordResponse( - FetchRecordReq fetchRecord, SplitReadResult readResult) - throws Exception { + FetchRecordReq fetchRecord, SplitReadResult readResult) throws Exception { RecordWithMeta recordResponse = new RecordWithMeta(); - MySqlSplit split = readResult.getSplit(); - MySqlSplitState currentSplitState = readResult.getSplitState(); + SourceSplit split = readResult.getSplit(); int count = 0; try { // Serialize records and add them to the response (collect from iterator) @@ -235,8 +240,9 @@ private RecordWithMeta buildRecordResponse( recordResponse.getRecords().addAll(serializedRecords); count += serializedRecords.size(); // update meta - Map lastMeta = RecordUtils.getBinlogPosition(element).getOffset(); - if (split.isBinlogSplit()) { + Map lastMeta = + RecordUtils.getBinlogPosition(element).getOffset(); + if (isBinlogSplit(split)) { lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); recordResponse.setMeta(lastMeta); } @@ -250,16 +256,14 @@ private RecordWithMeta buildRecordResponse( } // Set meta information - if (split.isSnapshotSplit() && currentSplitState != null) { - BinlogOffset highWatermark = - currentSplitState.asSnapshotSplitState().getHighWatermark(); - Map offsetRes = highWatermark.getOffset(); - offsetRes.put(SPLIT_ID, split.splitId()); + if (isSnapshotSplit(split) && readResult.getSplitState() != null) { + Map offsetRes = + extractSnapshotOffset(split, readResult.getSplitState()); recordResponse.setMeta(offsetRes); return recordResponse; } if (CollectionUtils.isEmpty(recordResponse.getRecords())) { - if (split.isBinlogSplit()) { + if (isBinlogSplit(split)) { Map offsetRes = extractBinlogOffset(readResult.getSplit()); recordResponse.setMeta(offsetRes); } else { @@ -285,10 +289,10 @@ private void refreshTableChanges(SourceRecord element, Long jobId) throws IOExce HistoryRecord historyRecord = RecordUtils.getHistoryRecord(element); Array tableChanges = historyRecord.document().getArray(HistoryRecord.Fields.TABLE_CHANGES); TableChanges changes = TABLE_CHANGE_SERIALIZER.deserialize(tableChanges, true); - Map tableChangeMap = jobRuntimeContext.getTableSchemas(); + Map tableChangeMap = this.getTableSchemas(); if (tableChangeMap == null) { tableChangeMap = new ConcurrentHashMap<>(); - jobRuntimeContext.setTableSchemas(tableChangeMap); + this.setTableSchemas(tableChangeMap); } for (TableChanges.TableChange tblChange : changes) { tableChangeMap.put(tblChange.getTable().id(), tblChange); @@ -420,7 +424,7 @@ private List startSplitChunks( List remainingTables = new ArrayList<>(); if (snapshotTable != null) { // need add database name - String database = config.get(LoadConstants.DATABASE); + String database = config.get(DataSourceConfigKeys.DATABASE); remainingTables.add(TableId.parse(database + "." + snapshotTable)); } List remainingSplits = new ArrayList<>(); @@ -453,7 +457,7 @@ private SplitRecords pollSplitRecordsWithSplit(MySqlSplit split, JobConfig jobCo } else if (split instanceof MySqlBinlogSplit) { currentReader = getBinlogSplitReader(jobConfig); } - jobRuntimeContext.setCurrentReader(currentReader); + this.setCurrentReader(currentReader); currentReader.submitSplit(split); currentSplitId = split.splitId(); // make split record available @@ -479,7 +483,7 @@ private SplitRecords pollSplitRecordsWithCurrentReader( private SnapshotSplitReader getSnapshotSplitReader(JobConfig config) { MySqlSourceConfig sourceConfig = getSourceConfig(config); - SnapshotSplitReader snapshotReader = jobRuntimeContext.getSnapshotReader(); + SnapshotSplitReader snapshotReader = this.getSnapshotReader(); if (snapshotReader == null) { final MySqlConnection jdbcConnection = DebeziumUtils.createMySqlConnection(sourceConfig); @@ -488,14 +492,14 @@ private SnapshotSplitReader getSnapshotSplitReader(JobConfig config) { final StatefulTaskContext statefulTaskContext = new StatefulTaskContext(sourceConfig, binaryLogClient, jdbcConnection); snapshotReader = new SnapshotSplitReader(statefulTaskContext, 0); - jobRuntimeContext.setSnapshotReader(snapshotReader); + this.setSnapshotReader(snapshotReader); } return snapshotReader; } private BinlogSplitReader getBinlogSplitReader(JobConfig config) { MySqlSourceConfig sourceConfig = getSourceConfig(config); - BinlogSplitReader binlogReader = jobRuntimeContext.getBinlogReader(); + BinlogSplitReader binlogReader = this.getBinlogReader(); if (binlogReader == null) { final MySqlConnection jdbcConnection = DebeziumUtils.createMySqlConnection(sourceConfig); @@ -504,37 +508,35 @@ private BinlogSplitReader getBinlogSplitReader(JobConfig config) { final StatefulTaskContext statefulTaskContext = new StatefulTaskContext(sourceConfig, binaryLogClient, jdbcConnection); binlogReader = new BinlogSplitReader(statefulTaskContext, 0); - jobRuntimeContext.setBinlogReader(binlogReader); + this.setBinlogReader(binlogReader); } return binlogReader; } private void closeSnapshotReader() { - SnapshotSplitReader reusedSnapshotReader = jobRuntimeContext.getSnapshotReader(); + SnapshotSplitReader reusedSnapshotReader = this.getSnapshotReader(); if (reusedSnapshotReader != null) { LOG.info( "Close snapshot reader {}", reusedSnapshotReader.getClass().getCanonicalName()); reusedSnapshotReader.close(); - DebeziumReader currentReader = - jobRuntimeContext.getCurrentReader(); + DebeziumReader currentReader = this.getCurrentReader(); if (reusedSnapshotReader == currentReader) { - jobRuntimeContext.setCurrentReader(null); + this.setCurrentReader(null); } - jobRuntimeContext.setSnapshotReader(null); + this.setSnapshotReader(null); } } private void closeBinlogReader() { - BinlogSplitReader reusedBinlogReader = jobRuntimeContext.getBinlogReader(); + BinlogSplitReader reusedBinlogReader = this.getBinlogReader(); if (reusedBinlogReader != null) { LOG.info("Close binlog reader {}", reusedBinlogReader.getClass().getCanonicalName()); reusedBinlogReader.close(); - DebeziumReader currentReader = - jobRuntimeContext.getCurrentReader(); + DebeziumReader currentReader = this.getCurrentReader(); if (reusedBinlogReader == currentReader) { - jobRuntimeContext.setCurrentReader(null); + this.setCurrentReader(null); } - jobRuntimeContext.setBinlogReader(null); + this.setBinlogReader(null); } } @@ -543,59 +545,43 @@ private MySqlSourceConfig getSourceConfig(JobConfig config) { } @Override - public Map extractSnapshotOffset(Object splitState, Object split) { - if (splitState == null) { - return null; - } + public Map extractSnapshotOffset(SourceSplit split, Object splitState) { + Preconditions.checkNotNull(split, "split is null"); + Preconditions.checkNotNull(splitState, "splitState is null"); MySqlSplitState mysqlSplitState = (MySqlSplitState) splitState; MySqlSplit mysqlSplit = (MySqlSplit) split; BinlogOffset highWatermark = mysqlSplitState.asSnapshotSplitState().getHighWatermark(); Map offsetRes = new HashMap<>(highWatermark.getOffset()); - if (mysqlSplit != null) { - offsetRes.put(SPLIT_ID, mysqlSplit.splitId()); - } + offsetRes.put(SPLIT_ID, mysqlSplit.splitId()); return offsetRes; } @Override - public Map extractBinlogOffset(Object split) { - if (split == null) { - return null; - } + public Map extractBinlogOffset(SourceSplit split) { + Preconditions.checkNotNull(split, "split is null"); MySqlSplit mysqlSplit = (MySqlSplit) split; Map offsetRes = mysqlSplit.asBinlogSplit().getStartingOffset().getOffset(); offsetRes.put(SPLIT_ID, BINLOG_SPLIT_ID); - // offsetRes.put(PURE_BINLOG_PHASE, String.valueOf(pureBinlogPhase)); return offsetRes; } @Override - public String getSplitId(Object split) { - if (split == null) { - return null; - } - MySqlSplit mysqlSplit = (MySqlSplit) split; - return mysqlSplit.splitId(); - } - - @Override - public boolean isBinlogSplit(Object split) { - if (split == null) { - return false; - } + public boolean isBinlogSplit(SourceSplit split) { + Preconditions.checkNotNull(split, "split is null"); MySqlSplit mysqlSplit = (MySqlSplit) split; return mysqlSplit.isBinlogSplit(); } @Override - public void close(Long jobId) { - jobRuntimeContext.close(); - LOG.info("Close source reader for job {}", jobId); + public boolean isSnapshotSplit(SourceSplit split) { + Preconditions.checkNotNull(split, "split is null"); + MySqlSplit mysqlSplit = (MySqlSplit) split; + return mysqlSplit.isSnapshotSplit(); } @Override public void finishSplitRecords() { - jobRuntimeContext.setCurrentSplitRecords(null); + this.setCurrentSplitRecords(null); } @Override @@ -629,10 +615,10 @@ public int compareOffset(CompareOffsetReq compareOffsetReq) { } private Map getTableSchemas(JobConfig config) { - Map schemas = jobRuntimeContext.getTableSchemas(); + Map schemas = this.getTableSchemas(); if (schemas == null) { schemas = discoverTableSchemas(config); - jobRuntimeContext.setTableSchemas(schemas); + this.setTableSchemas(schemas); } return schemas; } @@ -649,29 +635,20 @@ private Map discoverTableSchemas(JobConfig co } } - @Getter - @Setter - private static final class JobRuntimeContext { - private SnapshotSplitReader snapshotReader; - private BinlogSplitReader binlogReader; - private DebeziumReader currentReader; - private Map tableSchemas; - private SplitRecords currentSplitRecords; - private MySqlSplit currentSplit; - - private void close() { - if (snapshotReader != null) { - snapshotReader.close(); - snapshotReader = null; - } - if (binlogReader != null) { - binlogReader.close(); - binlogReader = null; - } - currentReader = null; - currentSplitRecords = null; - tableSchemas = null; + @Override + public void close(Long jobId) { + LOG.info("Close source reader for job {}", jobId); + if (snapshotReader != null) { + snapshotReader.close(); + snapshotReader = null; + } + if (binlogReader != null) { + binlogReader.close(); + binlogReader = null; } + currentReader = null; + currentSplitRecords = null; + tableSchemas = null; } /** diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java deleted file mode 100644 index a595bc20e76954..00000000000000 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/AbstractSourceSplit.java +++ /dev/null @@ -1,38 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.cdcclient.source.split; - -import java.io.Serializable; - -import lombok.AllArgsConstructor; -import lombok.Getter; -import lombok.NoArgsConstructor; -import lombok.Setter; - -@Getter -@Setter -@AllArgsConstructor -@NoArgsConstructor -public abstract class AbstractSourceSplit implements SourceSplit, Serializable { - private static final long serialVersionUID = 1L; - protected String splitId; - - public boolean snapshotSplit() { - return this instanceof SnapshotSplit; - } -} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java deleted file mode 100644 index b56bed007a6f89..00000000000000 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/BinlogSplit.java +++ /dev/null @@ -1,46 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.cdcclient.source.split; - -import java.util.List; -import java.util.Map; - -import lombok.EqualsAndHashCode; -import lombok.Getter; -import lombok.Setter; - -@Getter -@Setter -@EqualsAndHashCode(callSuper = true) -public class BinlogSplit extends AbstractSourceSplit { - private static final long serialVersionUID = 1L; - public static final String BINLOG_SPLIT_ID = "binlog-split"; - private Map startingOffset; - private Map endingOffset; - // binlog split meta, first binlog split requires - private List finishedSplits; - - public BinlogSplit() { - this.splitId = BINLOG_SPLIT_ID; - } - - public BinlogSplit(Map startingOffset) { - this.splitId = BINLOG_SPLIT_ID; - this.startingOffset = startingOffset; - } -} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java deleted file mode 100644 index 01bf7f0d7e67ef..00000000000000 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/split/SnapshotSplit.java +++ /dev/null @@ -1,56 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.cdcclient.source.split; - -import java.util.List; -import java.util.Map; - -import com.fasterxml.jackson.databind.ObjectMapper; -import lombok.EqualsAndHashCode; -import lombok.Getter; -import lombok.NoArgsConstructor; -import lombok.Setter; - -@Getter -@Setter -@EqualsAndHashCode(callSuper = true) -@NoArgsConstructor -public class SnapshotSplit extends AbstractSourceSplit { - private static final long serialVersionUID = 1L; - private static ObjectMapper objectMapper = new ObjectMapper(); - private String tableId; - private List splitKey; - private Object[] splitStart; - private Object[] splitEnd; - private Map highWatermark; - - public SnapshotSplit( - String splitId, - String tableId, - List splitKey, - Object[] splitStart, - Object[] splitEnd, - Map highWatermark) { - super(splitId); - this.tableId = tableId; - this.splitKey = splitKey; - this.splitStart = splitStart; - this.splitEnd = splitEnd; - this.highWatermark = highWatermark; - } -} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index 959cd1187bff66..80e299676e7ef2 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -17,8 +17,8 @@ package org.apache.doris.cdcclient.utils; -import org.apache.doris.cdcclient.constants.LoadConstants; import org.apache.doris.cdcclient.model.JobConfig; +import org.apache.doris.job.cdc.DataSourceConfigKeys; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; @@ -53,25 +53,26 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { MySqlSourceConfigFactory configFactory = new MySqlSourceConfigFactory(); ConnectionUrl cu = - ConnectionUrl.getConnectionUrlInstance(cdcConfig.get(LoadConstants.JDBC_URL), null); + ConnectionUrl.getConnectionUrlInstance( + cdcConfig.get(DataSourceConfigKeys.JDBC_URL), null); configFactory.hostname(cu.getMainHost().getHost()); configFactory.port(cu.getMainHost().getPort()); - configFactory.username(cdcConfig.get(LoadConstants.USER)); - configFactory.password(cdcConfig.get(LoadConstants.PASSWORD)); - String databaseName = cdcConfig.get(LoadConstants.DATABASE); + configFactory.username(cdcConfig.get(DataSourceConfigKeys.USER)); + configFactory.password(cdcConfig.get(DataSourceConfigKeys.PASSWORD)); + String databaseName = cdcConfig.get(DataSourceConfigKeys.DATABASE); configFactory.databaseList(databaseName); configFactory.serverId(getServerId(config.getJobId())); configFactory.includeSchemaChanges(false); - String includingTables = cdcConfig.get(LoadConstants.INCLUDE_TABLES); + String includingTables = cdcConfig.get(DataSourceConfigKeys.INCLUDE_TABLES); String[] includingTbls = Arrays.stream(includingTables.split(",")) .map(t -> databaseName + "." + t.trim()) .toArray(String[]::new); configFactory.tableList(includingTbls); - String excludingTables = cdcConfig.get(LoadConstants.EXCLUDE_TABLES); + String excludingTables = cdcConfig.get(DataSourceConfigKeys.EXCLUDE_TABLES); if (StringUtils.isNotEmpty(excludingTables)) { String excludingTbls = Arrays.stream(excludingTables.split(",")) @@ -81,17 +82,17 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { } // setting startMode - String startupMode = cdcConfig.get(LoadConstants.OFFSET); - if (LoadConstants.OFFSET_INITIAL.equalsIgnoreCase(startupMode)) { + String startupMode = cdcConfig.get(DataSourceConfigKeys.OFFSET); + if (DataSourceConfigKeys.OFFSET_INITIAL.equalsIgnoreCase(startupMode)) { // do not need set offset when initial // configFactory.startupOptions(StartupOptions.initial()); - } else if (LoadConstants.OFFSET_EARLIEST.equalsIgnoreCase(startupMode)) { + } else if (DataSourceConfigKeys.OFFSET_EARLIEST.equalsIgnoreCase(startupMode)) { configFactory.startupOptions(StartupOptions.earliest()); BinlogOffset binlogOffset = initializeEffectiveOffset( configFactory, StartupOptions.earliest().binlogOffset); configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); - } else if (LoadConstants.OFFSET_LATEST.equalsIgnoreCase(startupMode)) { + } else if (DataSourceConfigKeys.OFFSET_LATEST.equalsIgnoreCase(startupMode)) { configFactory.startupOptions(StartupOptions.latest()); BinlogOffset binlogOffset = initializeEffectiveOffset(configFactory, StartupOptions.latest().binlogOffset); @@ -125,8 +126,9 @@ public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { configFactory.jdbcProperties(jdbcProperteis); // configFactory.heartbeatInterval(Duration.ofMillis(1)); - if (cdcConfig.containsKey(LoadConstants.SPLIT_SIZE)) { - configFactory.splitSize(Integer.parseInt(cdcConfig.get(LoadConstants.SPLIT_SIZE))); + if (cdcConfig.containsKey(DataSourceConfigKeys.SPLIT_SIZE)) { + configFactory.splitSize( + Integer.parseInt(cdcConfig.get(DataSourceConfigKeys.SPLIT_SIZE))); } return configFactory.createConfig(0); @@ -166,7 +168,7 @@ private static Map toStringMap(String json) { try { return mapper.readValue(json, new TypeReference>() {}); } catch (JsonProcessingException e) { - return null; // 或抛异常,按你需要调整 + return null; } } } From 3f8813d3e0298bc2932f9c6b551c53c4baf31b2d Mon Sep 17 00:00:00 2001 From: JNSimba <676366545@qq.com> Date: Thu, 11 Dec 2025 16:43:10 +0800 Subject: [PATCH 23/27] extend to entity fe-common --- be/src/runtime/cdc_client_manager.cpp | 120 +++++++++++++++++++------- be/src/runtime/cdc_client_manager.h | 13 ++- be/src/runtime/exec_env.h | 6 +- be/src/runtime/exec_env_init.cpp | 2 +- build.sh | 2 +- fs_brokers/cdc_client/build.sh | 36 ++++++++ 6 files changed, 141 insertions(+), 38 deletions(-) create mode 100755 fs_brokers/cdc_client/build.sh diff --git a/be/src/runtime/cdc_client_manager.cpp b/be/src/runtime/cdc_client_manager.cpp index 08d128eac32162..75801052084db0 100644 --- a/be/src/runtime/cdc_client_manager.cpp +++ b/be/src/runtime/cdc_client_manager.cpp @@ -28,7 +28,11 @@ #include #endif +#include +#include +#include #include +#include #include "common/config.h" #include "common/logging.h" @@ -75,11 +79,79 @@ Status check_cdc_client_health(int retry_times, int sleep_time, std::string& hea return Status::OK(); } -// Start CDC client process -Status start_cdc_client(PRequestCdcClientResult* result) { +} // anonymous namespace + +CdcClientMgr::CdcClientMgr() = default; + +CdcClientMgr::~CdcClientMgr() { + stop(); + LOG(INFO) << "CdcClientMgr is destroyed"; +} + +void CdcClientMgr::stop() { + pid_t pid = _child_pid.load(); + if (pid > 0) { + // Check if process is still alive + if (kill(pid, 0) == 0) { + LOG(INFO) << "Stopping CDC client process, pid=" << pid; + + // Send SIGTERM for graceful shutdown + if (kill(pid, SIGTERM) == 0) { + // Wait up to 10 seconds for graceful shutdown + for (int i = 0; i < 10; ++i) { + std::this_thread::sleep_for(std::chrono::seconds(1)); + if (kill(pid, 0) != 0) { + // Process has exited + _child_pid.store(0); + LOG(INFO) << "CDC client process stopped gracefully"; + return; + } + } + } + + // Force kill if still alive + if (kill(pid, 0) == 0) { + LOG(WARNING) << "Force killing CDC client process, pid=" << pid; + kill(pid, SIGKILL); + // Wait for process to exit + int status = 0; + waitpid(pid, &status, 0); + } + _child_pid.store(0); + } else { + // Process already dead + _child_pid.store(0); + } + } + + LOG(INFO) << "CdcClientMgr is stopped"; +} + +Status CdcClientMgr::start_cdc_client(PRequestCdcClientResult* result) { + std::lock_guard lock(_start_mutex); + + pid_t existing_pid = _child_pid.load(); + if (existing_pid > 0) { + // Check if process is still alive + if (kill(existing_pid, 0) == 0) { + std::string check_response; + auto check_st = check_cdc_client_health(1, 0, check_response); + if (check_st.ok()) { + LOG(INFO) << "cdc client already started, pid=" << existing_pid; + return Status::OK(); + } else { + // Process exists but not healthy, reset PID + LOG(WARNING) << "CDC client process exists but unhealthy, pid=" << existing_pid; + _child_pid.store(0); + } + } else { + // Process is dead, reset PID + _child_pid.store(0); + } + } + Status st = Status::OK(); - // Check DORIS_HOME environment variable const char* doris_home = getenv("DORIS_HOME"); if (!doris_home) { st = Status::InternalError("DORIS_HOME environment variable is not set"); @@ -89,7 +161,6 @@ Status start_cdc_client(PRequestCdcClientResult* result) { return st; } - // Check LOG_DIR environment variable const char* log_dir = getenv("LOG_DIR"); if (!log_dir) { st = Status::InternalError("LOG_DIR environment variable is not set"); @@ -143,7 +214,6 @@ Status start_cdc_client(PRequestCdcClientResult* result) { sigaction(SIGCHLD, &act, NULL); LOG(INFO) << "Start to fork cdc client process with " << path; - // If has a forked process, the child process fails to start and will automatically exit pid_t pid = ::fork(); if (pid < 0) { // Fork failed @@ -153,13 +223,13 @@ Status start_cdc_client(PRequestCdcClientResult* result) { } return st; } else if (pid == 0) { + // Child process // When the parent process is killed, the child process also needs to exit #ifndef __APPLE__ prctl(PR_SET_PDEATHSIG, SIGKILL); #endif - LOG(INFO) << "Cdc client child process ready to start, " << pid - << ", response=" << std::endl; + LOG(INFO) << "Cdc client child process ready to start"; std::cout << "Cdc client child process ready to start." << std::endl; std::string java_bin = path + "/bin/java"; // java -jar -Dlog.path=xx cdc-client.jar --server.port=9096 --backend.http.port=8040 @@ -168,40 +238,30 @@ Status start_cdc_client(PRequestCdcClientResult* result) { std::cerr << "Cdc client child process error." << std::endl; exit(1); } else { + // Parent process: save PID and wait for startup + _child_pid.store(pid); + // Waiting for cdc to start, failed after more than 30 seconds std::string health_response; Status status = check_cdc_client_health(5, 6, health_response); if (!status.ok()) { - LOG(ERROR) << "Failed to start cdc client process, status=" << status.to_string() - << ", response=" << health_response; + // Reset PID if startup failed + _child_pid.store(0); st = Status::InternalError("Start cdc client failed."); if (result) { st.to_protobuf(result->mutable_status()); } } else { - LOG(INFO) << "Start cdc client success, status=" << status.to_string() - << ", response=" << health_response; + LOG(INFO) << "Start cdc client success, pid=" << pid + << ", status=" << status.to_string() << ", response=" << health_response; } } return st; } -} // anonymous namespace - -CdcClientManager::CdcClientManager() = default; - -CdcClientManager::~CdcClientManager() { - stop(); - LOG(INFO) << "CdcClientManager is destroyed"; -} - -void CdcClientManager::stop() { - LOG(INFO) << "CdcClientManager is stopped"; -} - -void CdcClientManager::request_cdc_client_impl(const PRequestCdcClientRequest* request, - PRequestCdcClientResult* result, - google::protobuf::Closure* done) { +void CdcClientMgr::request_cdc_client_impl(const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) { VLOG_RPC << "request to cdc client, api " << request->api(); brpc::ClosureGuard closure_guard(done); @@ -219,9 +279,9 @@ void CdcClientManager::request_cdc_client_impl(const PRequestCdcClientRequest* r st.to_protobuf(result->mutable_status()); } -Status CdcClientManager::send_request_to_cdc_client(const std::string& api, - const std::string& params_body, - std::string* response) { +Status CdcClientMgr::send_request_to_cdc_client(const std::string& api, + const std::string& params_body, + std::string* response) { std::string remote_url_prefix = fmt::format("http://127.0.0.1:{}{}", doris::config::cdc_client_port, api); diff --git a/be/src/runtime/cdc_client_manager.h b/be/src/runtime/cdc_client_manager.h index cee65245638b03..04bbcf965eae61 100644 --- a/be/src/runtime/cdc_client_manager.h +++ b/be/src/runtime/cdc_client_manager.h @@ -19,6 +19,8 @@ #include +#include +#include #include #include "common/status.h" @@ -30,10 +32,10 @@ class RpcController; namespace doris { -class CdcClientManager { +class CdcClientMgr { public: - CdcClientManager(); - ~CdcClientManager(); + CdcClientMgr(); + ~CdcClientMgr(); void stop(); @@ -44,6 +46,11 @@ class CdcClientManager { private: Status send_request_to_cdc_client(const std::string& api, const std::string& params_body, std::string* response); + + Status start_cdc_client(PRequestCdcClientResult* result); + + std::mutex _start_mutex; + std::atomic _child_pid {0}; }; } // namespace doris diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index e4dfea1702395e..5f771f70cd06dc 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -118,7 +118,7 @@ class HeartbeatFlags; class FrontendServiceClient; class FileMetaCache; class GroupCommitMgr; -class CdcClientManager; +class CdcClientMgr; class TabletSchemaCache; class TabletColumnObjectPool; class UserFunctionCache; @@ -278,7 +278,7 @@ class ExecEnv { SmallFileMgr* small_file_mgr() { return _small_file_mgr; } doris::vectorized::SpillStreamManager* spill_stream_mgr() { return _spill_stream_mgr; } GroupCommitMgr* group_commit_mgr() { return _group_commit_mgr; } - CdcClientManager* cdc_client_mgr() { return _cdc_client_mgr; } + CdcClientMgr* cdc_client_mgr() { return _cdc_client_mgr; } const std::vector& store_paths() const { return _store_paths; } @@ -511,7 +511,7 @@ class ExecEnv { // ip:brpc_port -> frontend_indo std::map _frontends; GroupCommitMgr* _group_commit_mgr = nullptr; - CdcClientManager* _cdc_client_mgr = nullptr; + CdcClientMgr* _cdc_client_mgr = nullptr; // Maybe we should use unique_ptr, but it need complete type, which means we need // to include many headers, and for some cpp file that do not need class like TabletSchemaCache, diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index c70ee3e663cd5c..3d68eb68bf240f 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -336,7 +336,7 @@ Status ExecEnv::_init(const std::vector& store_paths, RETURN_IF_ERROR(_routine_load_task_executor->init(MemInfo::mem_limit())); _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); _group_commit_mgr = new GroupCommitMgr(this); - _cdc_client_mgr = new CdcClientManager(); + _cdc_client_mgr = new CdcClientMgr(); _memtable_memory_limiter = std::make_unique(); _load_stream_map_pool = std::make_unique(); _delta_writer_v2_pool = std::make_unique(); diff --git a/build.sh b/build.sh index 0ba52c6c5de3e3..ef848701565051 100755 --- a/build.sh +++ b/build.sh @@ -1014,7 +1014,7 @@ fi if [[ "${BUILD_BE_CDC_CLIENT}" -eq 1 ]]; then install -d "${DORIS_OUTPUT}/be/lib/cdc_client" cd "${DORIS_HOME}/fs_brokers/cdc_client" - "${MVN_CMD}" package -DskipTests + ./build.sh rm -rf "${DORIS_OUTPUT}/be/lib/cdc_client"/* cp -r -p "${DORIS_HOME}/fs_brokers/cdc_client/target/cdc-client.jar" "${DORIS_OUTPUT}/be/lib/cdc_client/" cd "${DORIS_HOME}" diff --git a/fs_brokers/cdc_client/build.sh b/fs_brokers/cdc_client/build.sh new file mode 100755 index 00000000000000..52d9683baf0ee3 --- /dev/null +++ b/fs_brokers/cdc_client/build.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +set -eo pipefail + +ROOT="$(cd "$(dirname "${BASH_SOURCE[0]}")" &>/dev/null && pwd)" + +export DORIS_HOME="${ROOT}/../.." +. "${DORIS_HOME}/env.sh" + +export CDC_CLIENT_HOME="${ROOT}" + +cd "${DORIS_HOME}/fe" +"${MVN_CMD}" install -pl fe-common -Dskip.doc=true -DskipTests + + +echo "Install cdc client..." +cd "${CDC_CLIENT_HOME}" +"${MVN_CMD}" package -DskipTests + +echo "Finished" From cfff8f96dce9c0b97d2d91d6949ee1be7d5c9917 Mon Sep 17 00:00:00 2001 From: wudi Date: Thu, 11 Dec 2025 18:12:09 +0800 Subject: [PATCH 24/27] extend fe model to common --- .../job/cdc/request/CompareOffsetRequest.java | 24 ++++++--- .../job/cdc/request/FetchRecordRequest.java | 4 +- .../cdc/request/FetchTableSplitsRequest.java | 18 +++++-- .../doris/job/cdc/request/JobBaseConfig.java | 8 +-- .../job/cdc/request/JobBaseRecordRequest.java | 10 ++-- .../job/cdc/request/WriteRecordRequest.java | 4 +- .../cdc/{ => split}/AbstractSourceSplit.java | 2 +- .../job/cdc/{ => split}/BinlogSplit.java | 2 +- .../job/cdc/{ => split}/SnapshotSplit.java | 2 +- .../streaming/StreamingMultiTblTask.java | 38 +++++++------ .../doris/job/offset/jdbc/JdbcOffset.java | 4 +- .../offset/jdbc/JdbcSourceOffsetProvider.java | 54 ++++++------------- .../doris/job/util/StreamingJobUtils.java | 2 +- .../apache/doris/cdcclient/common/Env.java | 4 +- .../controller/ClientController.java | 24 ++++----- .../model/response/WriteMetaResp.java | 27 ---------- .../service/PipelineCoordinator.java | 46 ++++++++-------- .../cdcclient/source/reader/SourceReader.java | 22 ++++---- .../reader/mysql/MySqlSourceReader.java | 53 +++++++++--------- .../doris/cdcclient/utils/ConfigUtil.java | 4 +- 20 files changed, 163 insertions(+), 189 deletions(-) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java => fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/CompareOffsetRequest.java (63%) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchRecordReq.java => fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchRecordRequest.java (91%) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java => fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchTableSplitsRequest.java (67%) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java => fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseConfig.java (93%) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java => fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseRecordRequest.java (87%) rename fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java => fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/WriteRecordRequest.java (92%) rename fe/fe-common/src/main/java/org/apache/doris/job/cdc/{ => split}/AbstractSourceSplit.java (96%) rename fe/fe-common/src/main/java/org/apache/doris/job/cdc/{ => split}/BinlogSplit.java (97%) rename fe/fe-common/src/main/java/org/apache/doris/job/cdc/{ => split}/SnapshotSplit.java (98%) delete mode 100644 fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/WriteMetaResp.java diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/CompareOffsetRequest.java similarity index 63% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/CompareOffsetRequest.java index f5c596142fca61..8449afd96281e1 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/CompareOffsetReq.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/CompareOffsetRequest.java @@ -15,18 +15,30 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model.request; - -import org.apache.doris.cdcclient.model.JobConfig; - -import java.util.Map; +package org.apache.doris.job.cdc.request; +import lombok.AllArgsConstructor; import lombok.Getter; +import lombok.NoArgsConstructor; import lombok.Setter; +import java.util.Map; + @Getter @Setter -public class CompareOffsetReq extends JobConfig { +@AllArgsConstructor +@NoArgsConstructor +public class CompareOffsetRequest extends JobBaseConfig { private Map offsetFirst; private Map offsetSecond; + + public CompareOffsetRequest(Long jobId, + String sourceType, + Map sourceProperties, + Map offsetFirst, + Map offsetSecond) { + super(jobId, sourceType, sourceProperties); + this.offsetFirst = offsetFirst; + this.offsetSecond = offsetSecond; + } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchRecordReq.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchRecordRequest.java similarity index 91% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchRecordReq.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchRecordRequest.java index 3fcc474c076f48..f11539e68324d1 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchRecordReq.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchRecordRequest.java @@ -15,14 +15,14 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model.request; +package org.apache.doris.job.cdc.request; import lombok.Data; import lombok.EqualsAndHashCode; @Data @EqualsAndHashCode(callSuper = true) -public class FetchRecordReq extends JobBaseRecordReq { +public class FetchRecordRequest extends JobBaseRecordRequest { private boolean reload = true; private int fetchSize; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchTableSplitsRequest.java similarity index 67% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchTableSplitsRequest.java index 957abb198e3e07..f855e373958687 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/FetchTableSplitsReq.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchTableSplitsRequest.java @@ -15,17 +15,27 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model.request; - -import org.apache.doris.cdcclient.model.JobConfig; +package org.apache.doris.job.cdc.request; +import lombok.AllArgsConstructor; import lombok.EqualsAndHashCode; import lombok.Getter; +import lombok.NoArgsConstructor; import lombok.Setter; +import java.util.Map; + @Getter @Setter @EqualsAndHashCode(callSuper = true) -public class FetchTableSplitsReq extends JobConfig { +@AllArgsConstructor +@NoArgsConstructor +public class FetchTableSplitsRequest extends JobBaseConfig { private String snapshotTable; + + public FetchTableSplitsRequest(Long jobId, String name, + Map sourceProperties, String snapshotTable) { + super(jobId, name, sourceProperties); + this.snapshotTable = snapshotTable; + } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseConfig.java similarity index 93% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseConfig.java index fe29e280be24ee..bfdbf6a34558a9 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/JobConfig.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseConfig.java @@ -15,18 +15,18 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model; - -import java.util.Map; +package org.apache.doris.job.cdc.request; import lombok.AllArgsConstructor; import lombok.Data; import lombok.NoArgsConstructor; +import java.util.Map; + @Data @AllArgsConstructor @NoArgsConstructor -public class JobConfig { +public class JobBaseConfig { private Long jobId; private String dataSource; private Map config; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseRecordRequest.java similarity index 87% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseRecordRequest.java index f84bdf8f11ce37..a9a1be374dbede 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/JobBaseRecordReq.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseRecordRequest.java @@ -15,20 +15,18 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model.request; - -import org.apache.doris.cdcclient.model.JobConfig; - -import java.util.Map; +package org.apache.doris.job.cdc.request; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.Setter; +import java.util.Map; + @Getter @Setter @EqualsAndHashCode(callSuper = true) -public abstract class JobBaseRecordReq extends JobConfig { +public abstract class JobBaseRecordRequest extends JobBaseConfig { protected Map meta; public abstract boolean isReload(); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/WriteRecordRequest.java similarity index 92% rename from fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/WriteRecordRequest.java index 79c34028be02a1..a75edfcf7fb718 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/request/WriteRecordReq.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/WriteRecordRequest.java @@ -15,14 +15,14 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.cdcclient.model.request; +package org.apache.doris.job.cdc.request; import lombok.Data; import lombok.EqualsAndHashCode; @Data @EqualsAndHashCode(callSuper = true) -public class WriteRecordReq extends JobBaseRecordReq { +public class WriteRecordRequest extends JobBaseRecordRequest { private long maxInterval; private String targetDb; private String token; diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/AbstractSourceSplit.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/AbstractSourceSplit.java similarity index 96% rename from fe/fe-common/src/main/java/org/apache/doris/job/cdc/AbstractSourceSplit.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/AbstractSourceSplit.java index c7cb3c85f905c7..ab8fda340cf65e 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/AbstractSourceSplit.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/AbstractSourceSplit.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.job.cdc; +package org.apache.doris.job.cdc.split; import lombok.AllArgsConstructor; import lombok.Getter; diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/BinlogSplit.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/BinlogSplit.java similarity index 97% rename from fe/fe-common/src/main/java/org/apache/doris/job/cdc/BinlogSplit.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/BinlogSplit.java index 6dc7fc35247762..169c68e2b4499f 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/BinlogSplit.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/BinlogSplit.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.job.cdc; +package org.apache.doris.job.cdc.split; import com.google.gson.Gson; import lombok.EqualsAndHashCode; diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/SnapshotSplit.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/SnapshotSplit.java similarity index 98% rename from fe/fe-common/src/main/java/org/apache/doris/job/cdc/SnapshotSplit.java rename to fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/SnapshotSplit.java index 44977584d79204..99a84d1aaeb717 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/SnapshotSplit.java +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/SnapshotSplit.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.job.cdc; +package org.apache.doris.job.cdc.split; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java index 4a711764f569e2..c7adb6ad9c545d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java @@ -24,8 +24,9 @@ import org.apache.doris.httpv2.rest.RestApiStatusCode; import org.apache.doris.httpv2.rest.StreamingJobAction.CommitOffsetRequest; import org.apache.doris.job.base.Job; -import org.apache.doris.job.cdc.BinlogSplit; -import org.apache.doris.job.cdc.SnapshotSplit; +import org.apache.doris.job.cdc.request.WriteRecordRequest; +import org.apache.doris.job.cdc.split.BinlogSplit; +import org.apache.doris.job.cdc.split.SnapshotSplit; import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.exception.JobException; @@ -109,7 +110,7 @@ public void run() throws JobException { private void sendWriteRequest() throws JobException { Backend backend = StreamingJobUtils.selectBackend(jobId); - Map params = buildRequestParams(); + WriteRecordRequest params = buildRequestParams(); InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() .setApi("/api/writeRecords") .setParams(new Gson().toJson(params)).build(); @@ -160,21 +161,24 @@ private String getToken() throws JobException { return token; } - private Map buildRequestParams() throws JobException { + private WriteRecordRequest buildRequestParams() throws JobException { JdbcOffset offset = (JdbcOffset) runningOffset; - Map params = new HashMap<>(); - params.put("jobId", getJobId()); - params.put("labelName", getLabelName()); - params.put("dataSource", dataSourceType); - params.put("meta", offset.getSplit()); - params.put("config", sourceProperties); - params.put("targetDb", targetDb); - params.put("token", getToken()); - params.put("taskId", getTaskId()); - params.put("frontendAddress", - Env.getCurrentEnv().getMasterHost() + ":" + Env.getCurrentEnv().getMasterHttpPort()); - params.put("maxInterval", jobProperties.getMaxIntervalSecond()); - return params; + WriteRecordRequest request = new WriteRecordRequest(); + request.setJobId(getJobId()); + request.setConfig(sourceProperties); + request.setDataSource(dataSourceType.name()); + + request.setTaskId(getTaskId() + ""); + request.setToken(getToken()); + request.setTargetDb(targetDb); + Map splitMeta = objectMapper.convertValue(offset.getSplit(), + new TypeReference>() { + }); + request.setMeta(splitMeta); + String feAddr = Env.getCurrentEnv().getMasterHost() + ":" + Env.getCurrentEnv().getMasterHttpPort(); + request.setFrontendAddress(feAddr); + request.setMaxInterval(jobProperties.getMaxIntervalSecond()); + return request; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java index beba9f5d7d2b6f..83ad4314d8b85b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -17,8 +17,8 @@ package org.apache.doris.job.offset.jdbc; -import org.apache.doris.job.cdc.AbstractSourceSplit; -import org.apache.doris.job.cdc.BinlogSplit; +import org.apache.doris.job.cdc.split.AbstractSourceSplit; +import org.apache.doris.job.cdc.split.BinlogSplit; import org.apache.doris.job.offset.Offset; import com.google.gson.Gson; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index d5b29e581d4e06..c9a5fed1d5f323 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -18,10 +18,13 @@ package org.apache.doris.job.offset.jdbc; import org.apache.doris.httpv2.entity.ResponseBody; -import org.apache.doris.job.cdc.AbstractSourceSplit; -import org.apache.doris.job.cdc.BinlogSplit; import org.apache.doris.job.cdc.DataSourceConfigKeys; -import org.apache.doris.job.cdc.SnapshotSplit; +import org.apache.doris.job.cdc.request.CompareOffsetRequest; +import org.apache.doris.job.cdc.request.FetchTableSplitsRequest; +import org.apache.doris.job.cdc.request.JobBaseConfig; +import org.apache.doris.job.cdc.split.AbstractSourceSplit; +import org.apache.doris.job.cdc.split.BinlogSplit; +import org.apache.doris.job.cdc.split.SnapshotSplit; import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; @@ -119,14 +122,7 @@ public String getShowCurrentOffset() { if (this.currentOffset != null) { AbstractSourceSplit split = this.currentOffset.getSplit(); if (split.snapshotSplit()) { - SnapshotSplit snsplit = (SnapshotSplit) split; - Map splitShow = new HashMap<>(); - splitShow.put("splitId", snsplit.getSplitId()); - splitShow.put("tableId", snsplit.getTableId()); - splitShow.put("splitKey", snsplit.getSplitKey()); - splitShow.put("splitStart", snsplit.getSplitStart()); - splitShow.put("splitEnd", snsplit.getSplitEnd()); - return new Gson().toJson(splitShow); + return new Gson().toJson(split); } else { BinlogSplit binlogSplit = (BinlogSplit) split; HashMap showMap = new HashMap<>(); @@ -187,10 +183,10 @@ public void updateOffset(Offset offset) { @Override public void fetchRemoteMeta(Map properties) throws Exception { Backend backend = StreamingJobUtils.selectBackend(jobId); - Map params = buildBaseParams(); + JobBaseConfig requestParams = new JobBaseConfig(getJobId(), sourceType.name(), sourceProperties); InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() .setApi("/api/fetchEndOffset") - .setParams(new Gson().toJson(params)).build(); + .setParams(new Gson().toJson(requestParams)).build(); TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); InternalService.PRequestCdcClientResult result = null; try { @@ -262,10 +258,11 @@ public boolean hasMoreDataToConsume() { private boolean compareOffset(Map offsetFirst, Map offsetSecond) throws JobException { Backend backend = StreamingJobUtils.selectBackend(jobId); - Map params = buildCompareOffsetParams(offsetFirst, offsetSecond); + CompareOffsetRequest requestParams = + new CompareOffsetRequest(getJobId(), sourceType.name(), sourceProperties, offsetFirst, offsetSecond); InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() .setApi("/api/compareOffset") - .setParams(new Gson().toJson(params)).build(); + .setParams(new Gson().toJson(requestParams)).build(); TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); InternalService.PRequestCdcClientResult result = null; try { @@ -447,10 +444,11 @@ private void saveChunkMeta(Map> tableSplits) throws private List requestTableSplits(String table) throws JobException { Backend backend = StreamingJobUtils.selectBackend(jobId); - Map params = buildSplitParams(table); + FetchTableSplitsRequest requestParams = + new FetchTableSplitsRequest(getJobId(), sourceType.name(), sourceProperties, table); InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() .setApi("/api/fetchSplits") - .setParams(new Gson().toJson(params)).build(); + .setParams(new Gson().toJson(requestParams)).build(); TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); InternalService.PRequestCdcClientResult result = null; try { @@ -483,28 +481,6 @@ private List requestTableSplits(String table) throws JobException } } - private Map buildBaseParams() { - Map params = new HashMap<>(); - params.put("jobId", getJobId()); - params.put("dataSource", sourceType); - params.put("config", sourceProperties); - return params; - } - - private Map buildCompareOffsetParams(Map offsetFirst, - Map offsetSecond) { - Map params = buildBaseParams(); - params.put("offsetFirst", offsetFirst); - params.put("offsetSecond", offsetSecond); - return params; - } - - private Map buildSplitParams(String table) { - Map params = buildBaseParams(); - params.put("snapshotTable", table); - return params; - } - private boolean checkNeedSplitChunks(Map sourceProperties) { String startMode = sourceProperties.get(DataSourceConfigKeys.OFFSET); if (startMode == null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java index 2004073fe18ae8..4c0c9e93d93245 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java @@ -29,7 +29,7 @@ import org.apache.doris.datasource.jdbc.client.JdbcClientConfig; import org.apache.doris.datasource.jdbc.client.JdbcMySQLClient; import org.apache.doris.job.cdc.DataSourceConfigKeys; -import org.apache.doris.job.cdc.SnapshotSplit; +import org.apache.doris.job.cdc.split.SnapshotSplit; import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.exception.JobException; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java index be8bf466abe8f2..9596fd45e7052d 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java @@ -17,10 +17,10 @@ package org.apache.doris.cdcclient.common; -import org.apache.doris.cdcclient.model.JobConfig; import org.apache.doris.cdcclient.source.factory.DataSource; import org.apache.doris.cdcclient.source.factory.SourceReaderFactory; import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.job.cdc.request.JobBaseConfig; import java.util.Locale; import java.util.Map; @@ -53,7 +53,7 @@ public static Env getCurrentEnv() { return INSTANCE; } - public SourceReader getReader(JobConfig jobConfig) { + public SourceReader getReader(JobBaseConfig jobConfig) { DataSource ds = resolveDataSource(jobConfig.getDataSource()); Env manager = Env.getCurrentEnv(); return manager.getOrCreateReader(jobConfig.getJobId(), ds, jobConfig.getConfig()); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java index ca3fd2e5c61870..916b461e72dedf 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -18,14 +18,14 @@ package org.apache.doris.cdcclient.controller; import org.apache.doris.cdcclient.common.Env; -import org.apache.doris.cdcclient.model.JobConfig; -import org.apache.doris.cdcclient.model.request.CompareOffsetReq; -import org.apache.doris.cdcclient.model.request.FetchRecordReq; -import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; -import org.apache.doris.cdcclient.model.request.WriteRecordReq; import org.apache.doris.cdcclient.model.rest.RestResponse; import org.apache.doris.cdcclient.service.PipelineCoordinator; import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.job.cdc.request.CompareOffsetRequest; +import org.apache.doris.job.cdc.request.FetchRecordRequest; +import org.apache.doris.job.cdc.request.FetchTableSplitsRequest; +import org.apache.doris.job.cdc.request.JobBaseConfig; +import org.apache.doris.job.cdc.request.WriteRecordRequest; import java.util.List; @@ -46,7 +46,7 @@ public class ClientController { /** Fetch source splits for snapshot */ @RequestMapping(path = "/api/fetchSplits", method = RequestMethod.POST) - public Object fetchSplits(@RequestBody FetchTableSplitsReq ftsReq) { + public Object fetchSplits(@RequestBody FetchTableSplitsRequest ftsReq) { try { SourceReader reader = Env.getCurrentEnv().getReader(ftsReq); List splits = reader.getSourceSplits(ftsReq); @@ -59,7 +59,7 @@ public Object fetchSplits(@RequestBody FetchTableSplitsReq ftsReq) { /** Fetch records from source reader */ @RequestMapping(path = "/api/fetchRecords", method = RequestMethod.POST) - public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { + public Object fetchRecords(@RequestBody FetchRecordRequest recordReq) { try { SourceReader reader = Env.getCurrentEnv().getReader(recordReq); return RestResponse.success(reader.read(recordReq)); @@ -71,7 +71,7 @@ public Object fetchRecords(@RequestBody FetchRecordReq recordReq) { /** Fetch records from source reader and Write records to backend */ @RequestMapping(path = "/api/writeRecords", method = RequestMethod.POST) - public Object writeRecord(@RequestBody WriteRecordReq recordReq) { + public Object writeRecord(@RequestBody WriteRecordRequest recordReq) { LOG.info( "Received write record request for jobId={}, taskId={}, meta={}", recordReq.getJobId(), @@ -83,16 +83,16 @@ public Object writeRecord(@RequestBody WriteRecordReq recordReq) { /** Fetch lastest end meta */ @RequestMapping(path = "/api/fetchEndOffset", method = RequestMethod.POST) - public Object fetchEndOffset(@RequestBody JobConfig jobConfig) { + public Object fetchEndOffset(@RequestBody JobBaseConfig jobConfig) { SourceReader reader = Env.getCurrentEnv().getReader(jobConfig); return RestResponse.success(reader.getEndOffset(jobConfig)); } /** compare datasource Binlog Offset */ @RequestMapping(path = "/api/compareOffset", method = RequestMethod.POST) - public Object compareOffset(@RequestBody CompareOffsetReq compareOffsetReq) { - SourceReader reader = Env.getCurrentEnv().getReader(compareOffsetReq); - return RestResponse.success(reader.compareOffset(compareOffsetReq)); + public Object compareOffset(@RequestBody CompareOffsetRequest compareOffsetRequest) { + SourceReader reader = Env.getCurrentEnv().getReader(compareOffsetRequest); + return RestResponse.success(reader.compareOffset(compareOffsetRequest)); } /** Close job */ diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/WriteMetaResp.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/WriteMetaResp.java deleted file mode 100644 index a15b9078e3038a..00000000000000 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/WriteMetaResp.java +++ /dev/null @@ -1,27 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.cdcclient.model.response; - -import java.util.Map; - -import lombok.Data; - -@Data -public class WriteMetaResp { - private Map meta; -} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java index 8f9211a8e4ca43..bfaaed802f5e92 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -19,12 +19,12 @@ import org.apache.doris.cdcclient.common.Env; import org.apache.doris.cdcclient.exception.StreamLoadException; -import org.apache.doris.cdcclient.model.request.WriteRecordReq; import org.apache.doris.cdcclient.sink.DorisBatchStreamLoad; import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; import org.apache.doris.cdcclient.source.reader.SourceReader; import org.apache.doris.cdcclient.source.reader.SplitReadResult; +import org.apache.doris.job.cdc.request.WriteRecordRequest; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; @@ -78,27 +78,27 @@ public PipelineCoordinator() { new ThreadPoolExecutor.AbortPolicy()); } - public CompletableFuture writeRecordsAsync(WriteRecordReq writeRecordReq) { - Preconditions.checkNotNull(writeRecordReq.getToken(), "token must not be null"); - Preconditions.checkNotNull(writeRecordReq.getTaskId(), "taskId must not be null"); - Preconditions.checkNotNull(writeRecordReq.getTargetDb(), "targetDb must not be null"); + public CompletableFuture writeRecordsAsync(WriteRecordRequest writeRecordRequest) { + Preconditions.checkNotNull(writeRecordRequest.getToken(), "token must not be null"); + Preconditions.checkNotNull(writeRecordRequest.getTaskId(), "taskId must not be null"); + Preconditions.checkNotNull(writeRecordRequest.getTargetDb(), "targetDb must not be null"); return CompletableFuture.runAsync( () -> { try { LOG.info( "Start processing async write record, jobId={} taskId={}", - writeRecordReq.getJobId(), - writeRecordReq.getTaskId()); - writeRecords(writeRecordReq); + writeRecordRequest.getJobId(), + writeRecordRequest.getTaskId()); + writeRecords(writeRecordRequest); LOG.info( "Successfully processed async write record, jobId={} taskId={}", - writeRecordReq.getJobId(), - writeRecordReq.getTaskId()); + writeRecordRequest.getJobId(), + writeRecordRequest.getTaskId()); } catch (Exception ex) { LOG.error( "Failed to process async write record, jobId={} taskId={}", - writeRecordReq.getJobId(), - writeRecordReq.getTaskId(), + writeRecordRequest.getJobId(), + writeRecordRequest.getTaskId(), ex); } }, @@ -106,8 +106,8 @@ public CompletableFuture writeRecordsAsync(WriteRecordReq writeRecordReq) } /** Read data from SourceReader and write it to Doris, while returning meta information. */ - public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { - SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordReq); + public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception { + SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordRequest); DorisBatchStreamLoad batchStreamLoad = null; Map metaResponse = new HashMap<>(); boolean hasData = false; @@ -115,26 +115,26 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { long scannedBytes = 0L; SplitReadResult readResult = null; try { - readResult = sourceReader.readSplitRecords(writeRecordReq); + readResult = sourceReader.readSplitRecords(writeRecordRequest); batchStreamLoad = getOrCreateBatchStreamLoad( - writeRecordReq.getJobId(), writeRecordReq.getTargetDb()); - batchStreamLoad.setCurrentTaskId(writeRecordReq.getTaskId()); - batchStreamLoad.setFrontendAddress(writeRecordReq.getFrontendAddress()); - batchStreamLoad.setToken(writeRecordReq.getToken()); + writeRecordRequest.getJobId(), writeRecordRequest.getTargetDb()); + batchStreamLoad.setCurrentTaskId(writeRecordRequest.getTaskId()); + batchStreamLoad.setFrontendAddress(writeRecordRequest.getFrontendAddress()); + batchStreamLoad.setToken(writeRecordRequest.getToken()); // Record start time for maxInterval check long startTime = System.currentTimeMillis(); - long maxIntervalMillis = writeRecordReq.getMaxInterval() * 1000; + long maxIntervalMillis = writeRecordRequest.getMaxInterval() * 1000; // Use iterators to read and write. Iterator iterator = readResult.getRecordIterator(); while (iterator != null && iterator.hasNext()) { SourceRecord element = iterator.next(); List serializedRecords = - serializer.deserialize(writeRecordReq.getConfig(), element); + serializer.deserialize(writeRecordRequest.getConfig(), element); if (!CollectionUtils.isEmpty(serializedRecords)) { - String database = writeRecordReq.getTargetDb(); + String database = writeRecordRequest.getTargetDb(); String table = extractTable(element); hasData = true; for (String record : serializedRecords) { @@ -157,7 +157,7 @@ public void writeRecords(WriteRecordReq writeRecordReq) throws Exception { if (maxIntervalMillis > 0 && elapsedTime >= maxIntervalMillis) { LOG.info( "Max interval {} seconds reached, stopping data reading", - writeRecordReq.getMaxInterval()); + writeRecordRequest.getMaxInterval()); break; } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java index 4a319aba12bad9..b9bd8e94576201 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -17,13 +17,13 @@ package org.apache.doris.cdcclient.source.reader; -import org.apache.doris.cdcclient.model.JobConfig; -import org.apache.doris.cdcclient.model.request.CompareOffsetReq; -import org.apache.doris.cdcclient.model.request.FetchRecordReq; -import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; -import org.apache.doris.cdcclient.model.request.JobBaseRecordReq; import org.apache.doris.cdcclient.model.response.RecordWithMeta; -import org.apache.doris.job.cdc.AbstractSourceSplit; +import org.apache.doris.job.cdc.request.CompareOffsetRequest; +import org.apache.doris.job.cdc.request.FetchRecordRequest; +import org.apache.doris.job.cdc.request.FetchTableSplitsRequest; +import org.apache.doris.job.cdc.request.JobBaseConfig; +import org.apache.doris.job.cdc.request.JobBaseRecordRequest; +import org.apache.doris.job.cdc.split.AbstractSourceSplit; import org.apache.flink.api.connector.source.SourceSplit; @@ -36,13 +36,13 @@ public interface SourceReader { void initialize(); /** Divide the data to be read. For example: split mysql to chunks */ - List getSourceSplits(FetchTableSplitsReq config); + List getSourceSplits(FetchTableSplitsRequest config); /** Reading Data */ - RecordWithMeta read(FetchRecordReq meta) throws Exception; + RecordWithMeta read(FetchRecordRequest meta) throws Exception; /** Reading Data for split reader */ - SplitReadResult readSplitRecords(JobBaseRecordReq baseReq) throws Exception; + SplitReadResult readSplitRecords(JobBaseRecordRequest baseReq) throws Exception; /** Extract offset information from snapshot split state. */ Map extractSnapshotOffset(SourceSplit split, Object splitState); @@ -60,10 +60,10 @@ public interface SourceReader { void finishSplitRecords(); /** Get the end offset for the job */ - Map getEndOffset(JobConfig jobConfig); + Map getEndOffset(JobBaseConfig jobConfig); /** Compare the offsets */ - int compareOffset(CompareOffsetReq compareOffsetReq); + int compareOffset(CompareOffsetRequest compareOffsetRequest); /** Called when closing */ void close(Long jobId); diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index c60024ea2e0131..e630087caaf860 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -17,11 +17,6 @@ package org.apache.doris.cdcclient.source.reader.mysql; -import org.apache.doris.cdcclient.model.JobConfig; -import org.apache.doris.cdcclient.model.request.CompareOffsetReq; -import org.apache.doris.cdcclient.model.request.FetchRecordReq; -import org.apache.doris.cdcclient.model.request.FetchTableSplitsReq; -import org.apache.doris.cdcclient.model.request.JobBaseRecordReq; import org.apache.doris.cdcclient.model.response.RecordWithMeta; import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; @@ -29,10 +24,15 @@ import org.apache.doris.cdcclient.source.reader.SplitReadResult; import org.apache.doris.cdcclient.source.reader.SplitRecords; import org.apache.doris.cdcclient.utils.ConfigUtil; -import org.apache.doris.job.cdc.AbstractSourceSplit; -import org.apache.doris.job.cdc.BinlogSplit; import org.apache.doris.job.cdc.DataSourceConfigKeys; -import org.apache.doris.job.cdc.SnapshotSplit; +import org.apache.doris.job.cdc.request.CompareOffsetRequest; +import org.apache.doris.job.cdc.request.FetchRecordRequest; +import org.apache.doris.job.cdc.request.FetchTableSplitsRequest; +import org.apache.doris.job.cdc.request.JobBaseConfig; +import org.apache.doris.job.cdc.request.JobBaseRecordRequest; +import org.apache.doris.job.cdc.split.AbstractSourceSplit; +import org.apache.doris.job.cdc.split.BinlogSplit; +import org.apache.doris.job.cdc.split.SnapshotSplit; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.connector.source.SourceSplit; @@ -114,7 +114,7 @@ public MySqlSourceReader() { public void initialize() {} @Override - public List getSourceSplits(FetchTableSplitsReq ftsReq) { + public List getSourceSplits(FetchTableSplitsRequest ftsReq) { MySqlSourceConfig sourceConfig = getSourceConfig(ftsReq); StartupMode startupMode = sourceConfig.getStartupOptions().startupMode; List remainingSnapshotSplits = new ArrayList<>(); @@ -160,14 +160,14 @@ public List getSourceSplits(FetchTableSplitsReq ftsReq) { * reset binlogSplitReader and submit split. */ @Override - public RecordWithMeta read(FetchRecordReq fetchRecord) throws Exception { + public RecordWithMeta read(FetchRecordRequest fetchRecord) throws Exception { SplitReadResult readResult = readSplitRecords(fetchRecord); return buildRecordResponse(fetchRecord, readResult); } /** read split records. */ @Override - public SplitReadResult readSplitRecords(JobBaseRecordReq baseReq) throws Exception { + public SplitReadResult readSplitRecords(JobBaseRecordRequest baseReq) throws Exception { Map offsetMeta = baseReq.getMeta(); if (offsetMeta == null || offsetMeta.isEmpty()) { throw new RuntimeException("miss meta offset"); @@ -225,7 +225,7 @@ public SplitReadResult readSplitRecords(JobBaseRecordReq baseReq) throws Excepti /** build RecordWithMeta */ private RecordWithMeta buildRecordResponse( - FetchRecordReq fetchRecord, SplitReadResult readResult) throws Exception { + FetchRecordRequest fetchRecord, SplitReadResult readResult) throws Exception { RecordWithMeta recordResponse = new RecordWithMeta(); SourceSplit split = readResult.getSplit(); int count = 0; @@ -300,7 +300,8 @@ private void refreshTableChanges(SourceRecord element, Long jobId) throws IOExce } private Tuple2 createMySqlSplit( - Map offsetMeta, JobConfig jobConfig) throws JsonProcessingException { + Map offsetMeta, JobBaseConfig jobConfig) + throws JsonProcessingException { Tuple2 splitRes = null; String splitId = String.valueOf(offsetMeta.get(SPLIT_ID)); if (!BINLOG_SPLIT_ID.equals(splitId)) { @@ -312,8 +313,8 @@ private Tuple2 createMySqlSplit( return splitRes; } - private MySqlSnapshotSplit createSnapshotSplit(Map offset, JobConfig jobConfig) - throws JsonProcessingException { + private MySqlSnapshotSplit createSnapshotSplit( + Map offset, JobBaseConfig jobConfig) throws JsonProcessingException { SnapshotSplit snapshotSplit = objectMapper.convertValue(offset, SnapshotSplit.class); TableId tableId = TableId.parse(snapshotSplit.getTableId()); Object[] splitStart = snapshotSplit.getSplitStart(); @@ -340,7 +341,7 @@ private MySqlSnapshotSplit createSnapshotSplit(Map offset, JobCo } private Tuple2 createBinlogSplit( - Map meta, JobConfig config) { + Map meta, JobBaseConfig config) { MySqlSourceConfig sourceConfig = getSourceConfig(config); BinlogOffset offsetConfig = null; if (sourceConfig.getStartupOptions() != null) { @@ -445,7 +446,7 @@ private List startSplitChunks( return remainingSplits; } - private SplitRecords pollSplitRecordsWithSplit(MySqlSplit split, JobConfig jobConfig) + private SplitRecords pollSplitRecordsWithSplit(MySqlSplit split, JobBaseConfig jobConfig) throws Exception { Preconditions.checkState(split != null, "split is null"); Iterator dataIt = null; @@ -481,7 +482,7 @@ private SplitRecords pollSplitRecordsWithCurrentReader( } } - private SnapshotSplitReader getSnapshotSplitReader(JobConfig config) { + private SnapshotSplitReader getSnapshotSplitReader(JobBaseConfig config) { MySqlSourceConfig sourceConfig = getSourceConfig(config); SnapshotSplitReader snapshotReader = this.getSnapshotReader(); if (snapshotReader == null) { @@ -497,7 +498,7 @@ private SnapshotSplitReader getSnapshotSplitReader(JobConfig config) { return snapshotReader; } - private BinlogSplitReader getBinlogSplitReader(JobConfig config) { + private BinlogSplitReader getBinlogSplitReader(JobBaseConfig config) { MySqlSourceConfig sourceConfig = getSourceConfig(config); BinlogSplitReader binlogReader = this.getBinlogReader(); if (binlogReader == null) { @@ -540,7 +541,7 @@ private void closeBinlogReader() { } } - private MySqlSourceConfig getSourceConfig(JobConfig config) { + private MySqlSourceConfig getSourceConfig(JobBaseConfig config) { return ConfigUtil.generateMySqlConfig(config); } @@ -585,7 +586,7 @@ public void finishSplitRecords() { } @Override - public Map getEndOffset(JobConfig jobConfig) { + public Map getEndOffset(JobBaseConfig jobConfig) { MySqlSourceConfig sourceConfig = getSourceConfig(jobConfig); try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) { BinlogOffset binlogOffset = DebeziumUtils.currentBinlogOffset(jdbc); @@ -596,9 +597,9 @@ public Map getEndOffset(JobConfig jobConfig) { } @Override - public int compareOffset(CompareOffsetReq compareOffsetReq) { - Map offsetFirst = compareOffsetReq.getOffsetFirst(); - Map offsetSecond = compareOffsetReq.getOffsetSecond(); + public int compareOffset(CompareOffsetRequest compareOffsetRequest) { + Map offsetFirst = compareOffsetRequest.getOffsetFirst(); + Map offsetSecond = compareOffsetRequest.getOffsetSecond(); // make server id is equals String serverId1 = offsetFirst.get("server_id"); String serverId2 = offsetSecond.get("server_id"); @@ -614,7 +615,7 @@ public int compareOffset(CompareOffsetReq compareOffsetReq) { return binlogOffset1.compareTo(binlogOffset2); } - private Map getTableSchemas(JobConfig config) { + private Map getTableSchemas(JobBaseConfig config) { Map schemas = this.getTableSchemas(); if (schemas == null) { schemas = discoverTableSchemas(config); @@ -623,7 +624,7 @@ private Map getTableSchemas(JobConfig config) return schemas; } - private Map discoverTableSchemas(JobConfig config) { + private Map discoverTableSchemas(JobBaseConfig config) { MySqlSourceConfig sourceConfig = getSourceConfig(config); try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) { MySqlPartition partition = diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java index 80e299676e7ef2..51cd8828f53299 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -17,8 +17,8 @@ package org.apache.doris.cdcclient.utils; -import org.apache.doris.cdcclient.model.JobConfig; import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.cdc.request.JobBaseConfig; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; @@ -48,7 +48,7 @@ public static String getServerId(long jobId) { return String.valueOf(Math.abs(String.valueOf(jobId).hashCode())); } - public static MySqlSourceConfig generateMySqlConfig(JobConfig config) { + public static MySqlSourceConfig generateMySqlConfig(JobBaseConfig config) { Map cdcConfig = config.getConfig(); MySqlSourceConfigFactory configFactory = new MySqlSourceConfigFactory(); From 2dd280b48f3f20b09fe98cc59748540e1ed9e93c Mon Sep 17 00:00:00 2001 From: wudi Date: Mon, 15 Dec 2025 09:57:45 +0800 Subject: [PATCH 25/27] rename cdc clientmgr --- be/src/runtime/{cdc_client_manager.cpp => cdc_client_mgr.cpp} | 2 +- be/src/runtime/{cdc_client_manager.h => cdc_client_mgr.h} | 2 +- be/src/runtime/exec_env_init.cpp | 2 +- be/src/service/internal_service.cpp | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) rename be/src/runtime/{cdc_client_manager.cpp => cdc_client_mgr.cpp} (99%) rename be/src/runtime/{cdc_client_manager.h => cdc_client_mgr.h} (100%) diff --git a/be/src/runtime/cdc_client_manager.cpp b/be/src/runtime/cdc_client_mgr.cpp similarity index 99% rename from be/src/runtime/cdc_client_manager.cpp rename to be/src/runtime/cdc_client_mgr.cpp index 75801052084db0..67328ff2235766 100644 --- a/be/src/runtime/cdc_client_manager.cpp +++ b/be/src/runtime/cdc_client_mgr.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "runtime/cdc_client_manager.h" +#include "runtime/cdc_client_mgr.h" #include #include diff --git a/be/src/runtime/cdc_client_manager.h b/be/src/runtime/cdc_client_mgr.h similarity index 100% rename from be/src/runtime/cdc_client_manager.h rename to be/src/runtime/cdc_client_mgr.h index 04bbcf965eae61..6c055da08d0819 100644 --- a/be/src/runtime/cdc_client_manager.h +++ b/be/src/runtime/cdc_client_mgr.h @@ -43,12 +43,12 @@ class CdcClientMgr { void request_cdc_client_impl(const PRequestCdcClientRequest* request, PRequestCdcClientResult* result, google::protobuf::Closure* done); -private: Status send_request_to_cdc_client(const std::string& api, const std::string& params_body, std::string* response); Status start_cdc_client(PRequestCdcClientResult* result); +private: std::mutex _start_mutex; std::atomic _child_pid {0}; }; diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 3d68eb68bf240f..cb4ad78984b973 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -70,7 +70,7 @@ #include "pipeline/task_scheduler.h" #include "runtime/broker_mgr.h" #include "runtime/cache/result_cache.h" -#include "runtime/cdc_client_manager.h" +#include "runtime/cdc_client_mgr.h" #include "runtime/client_cache.h" #include "runtime/exec_env.h" #include "runtime/external_scan_context_mgr.h" diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 3c072d0b89d998..0043b5e0a17291 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -84,7 +84,7 @@ #include "olap/txn_manager.h" #include "olap/wal/wal_manager.h" #include "runtime/cache/result_cache.h" -#include "runtime/cdc_client_manager.h" +#include "runtime/cdc_client_mgr.h" #include "runtime/descriptors.h" #include "runtime/exec_env.h" #include "runtime/fold_constant_executor.h" From 84a87f331ece9e1cdd5f68e54b2479a65cce15c7 Mon Sep 17 00:00:00 2001 From: wudi Date: Mon, 15 Dec 2025 16:48:26 +0800 Subject: [PATCH 26/27] add create alter case --- .../org/apache/doris/nereids/DorisParser.g4 | 14 +- .../streaming/DataSourceConfigValidator.java | 79 ++++ .../insert/streaming/StreamingInsertJob.java | 14 +- .../offset/jdbc/JdbcSourceOffsetProvider.java | 2 +- .../doris/job/util/StreamingJobUtils.java | 4 + .../nereids/parser/LogicalPlanBuilder.java | 34 +- .../trees/plans/commands/AlterJobCommand.java | 130 +++++- .../plans/commands/CreateJobCommand.java | 8 + .../plans/commands/info/CreateJobInfo.java | 27 +- .../cdc/test_streaming_mysql_job.groovy | 3 +- ...st_streaming_mysql_job_create_alter.groovy | 430 ++++++++++++++++++ ...test_streaming_mysql_job_restart_fe.groovy | 118 +++++ 12 files changed, 829 insertions(+), 34 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_create_alter.groovy create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_restart_fe.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index ff3a336fe6e5f1..6e28c63bec2487 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -94,6 +94,12 @@ materializedViewStatement | CANCEL MATERIALIZED VIEW TASK taskId=INTEGER_VALUE ON mvName=multipartIdentifier #cancelMTMVTask | SHOW CREATE MATERIALIZED VIEW mvName=multipartIdentifier #showCreateMTMV ; + +jobFromToClause + : FROM sourceType=identifier LEFT_PAREN sourceProperties=propertyItemList RIGHT_PAREN + TO DATABASE targetDb=identifier (LEFT_PAREN targetProperties=propertyItemList RIGHT_PAREN)? + ; + supportedJobStatement : CREATE JOB label=multipartIdentifier jobProperties=propertyClause? ON (STREAMING | SCHEDULE( @@ -105,11 +111,11 @@ supportedJobStatement ) ) commentSpec? - (FROM sourceType=identifier LEFT_PAREN sourceProperties=propertyItemList RIGHT_PAREN - TO DATABASE targetDb=identifier (LEFT_PAREN targetProperties=propertyItemList RIGHT_PAREN)? - | DO supportedDmlStatement ) #createScheduledJob + (jobFromToClause | DO supportedDmlStatement ) #createScheduledJob | PAUSE JOB WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #pauseJob - | ALTER JOB (jobName=multipartIdentifier) (propertyClause | supportedDmlStatement | propertyClause supportedDmlStatement) #alterJob + | ALTER JOB (jobName=multipartIdentifier) + (propertyClause | supportedDmlStatement | propertyClause supportedDmlStatement + | jobFromToClause | propertyClause jobFromToClause) #alterJob | DROP JOB (IF EXISTS)? WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #dropJob | RESUME JOB WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #resumeJob | CANCEL TASK WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) AND (taskIdKey=identifier) EQ (taskIdValue=INTEGER_VALUE) #cancelJobTask diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java new file mode 100644 index 00000000000000..f8850dd9d70f19 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java @@ -0,0 +1,79 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.extensions.insert.streaming; + +import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.util.StreamingJobUtils; + +import com.google.common.collect.Sets; + +import java.util.Map; +import java.util.Set; + +public class DataSourceConfigValidator { + private static final Set ALLOW_SOURCE_KEYS = Sets.newHashSet( + DataSourceConfigKeys.JDBC_URL, + DataSourceConfigKeys.USER, + DataSourceConfigKeys.PASSWORD, + DataSourceConfigKeys.OFFSET, + DataSourceConfigKeys.DRIVER_URL, + DataSourceConfigKeys.DRIVER_CLASS, + DataSourceConfigKeys.DATABASE, + DataSourceConfigKeys.INCLUDE_TABLES, + DataSourceConfigKeys.EXCLUDE_TABLES + ); + + public static void validateSource(Map input) throws IllegalArgumentException { + for (Map.Entry entry : input.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + + if (!ALLOW_SOURCE_KEYS.contains(key)) { + throw new IllegalArgumentException("Unexpected key: '" + key + "'"); + } + + if (!isValidValue(key, value)) { + throw new IllegalArgumentException("Invalid value for key '" + key + "': " + value); + } + } + } + + public static void validateTarget(Map input) throws IllegalArgumentException { + for (Map.Entry entry : input.entrySet()) { + String key = entry.getKey(); + if (!key.startsWith(StreamingJobUtils.TABLE_PROPS_PREFIX)) { + throw new IllegalArgumentException("Only support target properties with prefix " + + StreamingJobUtils.TABLE_PROPS_PREFIX); + } + } + } + + private static boolean isValidValue(String key, String value) { + if (value == null || value.isEmpty()) { + return false; + } + + if (key.equals(DataSourceConfigKeys.OFFSET) + && !(value.equals(DataSourceConfigKeys.OFFSET_INITIAL) + || value.equals(DataSourceConfigKeys.OFFSET_LATEST))) { + return false; + } + return true; + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 835aa0240483ad..2a22f323d2d081 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -362,6 +362,18 @@ public void alterJob(AlterJobCommand alterJobCommand) throws AnalysisException, modifyPropertiesInternal(alterJobCommand.getProperties()); logParts.add("properties: " + alterJobCommand.getProperties()); } + + // update source properties + if (!alterJobCommand.getSourceProperties().isEmpty()) { + this.sourceProperties.putAll(alterJobCommand.getSourceProperties()); + logParts.add("source properties: " + alterJobCommand.getSourceProperties()); + } + + // update target properties + if (!alterJobCommand.getTargetProperties().isEmpty()) { + this.sourceProperties.putAll(alterJobCommand.getTargetProperties()); + logParts.add("target properties: " + alterJobCommand.getTargetProperties()); + } log.info("Alter streaming job {}, {}", getJobId(), String.join(", ", logParts)); } @@ -662,7 +674,7 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { */ private void modifyPropertiesInternal(Map inputProperties) throws AnalysisException, JobException { StreamingJobProperties inputStreamProps = new StreamingJobProperties(inputProperties); - if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty())) { + if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty()) && this.tvfType != null) { Offset offset = validateOffset(inputStreamProps.getOffsetProperty()); this.offsetProvider.updateOffset(offset); if (Config.isCloudMode()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index c9a5fed1d5f323..72a585c1b07fd1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -320,7 +320,7 @@ public Offset deserializeOffset(String offset) { @Override public Offset deserializeOffsetProperty(String offset) { - // todo: use for alter offset for job + // no need return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java index 4c0c9e93d93245..5772f740b3b905 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java @@ -302,6 +302,10 @@ public static List generateCreateTableCmds(String targetDb, CreateTableCommand createtblCmd = new CreateTableCommand(Optional.empty(), createtblInfo); createtblCmds.add(createtblCmd); } + if (createtblCmds.isEmpty()) { + throw new JobException("Can not found match table in database " + database); + } + return createtblCmds; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index a93e610e424e1a..43238de63aea93 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -244,6 +244,7 @@ import org.apache.doris.nereids.DorisParser.IntervalContext; import org.apache.doris.nereids.DorisParser.Is_not_null_predContext; import org.apache.doris.nereids.DorisParser.IsnullContext; +import org.apache.doris.nereids.DorisParser.JobFromToClauseContext; import org.apache.doris.nereids.DorisParser.JoinCriteriaContext; import org.apache.doris.nereids.DorisParser.JoinRelationContext; import org.apache.doris.nereids.DorisParser.KillQueryContext; @@ -1182,12 +1183,18 @@ public LogicalPlan visitCreateScheduledJob(DorisParser.CreateScheduledJobContext String comment = visitCommentSpec(ctx.commentSpec()); String executeSql = ctx.supportedDmlStatement() == null ? "" : getOriginSql(ctx.supportedDmlStatement()); - Optional sourceType = ctx.sourceType == null ? Optional.empty() : Optional.of(ctx.sourceType.getText()); - String targetDb = ctx.targetDb == null ? "" : ctx.targetDb.getText(); - Map sourceProperties = ctx.sourceProperties != null - ? Maps.newHashMap(visitPropertyItemList(ctx.sourceProperties)) : Maps.newHashMap(); - Map targetProperties = ctx.targetProperties != null - ? Maps.newHashMap(visitPropertyItemList(ctx.targetProperties)) : Maps.newHashMap(); + JobFromToClauseContext jobFromToClauseCtx = ctx.jobFromToClause(); + String sourceType = null; + String targetDb = null; + Map sourceProperties = Maps.newHashMap(); + Map targetProperties = Maps.newHashMap(); + if (jobFromToClauseCtx != null) { + sourceType = jobFromToClauseCtx.sourceType.getText(); + targetDb = jobFromToClauseCtx.targetDb == null ? "" : jobFromToClauseCtx.targetDb.getText(); + sourceProperties = Maps.newHashMap(visitPropertyItemList(jobFromToClauseCtx.sourceProperties)); + targetProperties = jobFromToClauseCtx.targetProperties != null + ? Maps.newHashMap(visitPropertyItemList(jobFromToClauseCtx.targetProperties)) : Maps.newHashMap(); + } CreateJobInfo createJobInfo = new CreateJobInfo(label, atTime, interval, intervalUnit, startTime, endsTime, immediateStartOptional, comment, executeSql, ctx.STREAMING() != null, jobProperties, sourceType, targetDb, sourceProperties, targetProperties); @@ -1205,7 +1212,20 @@ public LogicalPlan visitAlterJob(DorisParser.AlterJobContext ctx) { Map properties = ctx.propertyClause() != null ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); String executeSql = ctx.supportedDmlStatement() != null ? getOriginSql(ctx.supportedDmlStatement()) : ""; - return new AlterJobCommand(ctx.jobName.getText(), properties, executeSql); + String sourceType = null; + String targetDb = null; + Map sourceProperties = Maps.newHashMap(); + Map targetProperties = Maps.newHashMap(); + if (ctx.jobFromToClause() != null) { + sourceType = ctx.jobFromToClause().sourceType.getText(); + targetDb = ctx.jobFromToClause().targetDb == null ? "" : ctx.jobFromToClause().targetDb.getText(); + sourceProperties = Maps.newHashMap(visitPropertyItemList(ctx.jobFromToClause().sourceProperties)); + targetProperties = ctx.jobFromToClause().targetProperties != null + ? Maps.newHashMap(visitPropertyItemList(ctx.jobFromToClause().targetProperties)) + : Maps.newHashMap(); + } + return new AlterJobCommand(ctx.jobName.getText(), properties, + executeSql, sourceType, targetDb, sourceProperties, targetProperties); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index 6cbd17ced6c20a..584ee0848db2a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -23,7 +23,9 @@ import org.apache.doris.common.Pair; import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.base.JobExecuteType; +import org.apache.doris.job.cdc.DataSourceConfigKeys; import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.extensions.insert.streaming.DataSourceConfigValidator; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.nereids.analyzer.UnboundTVFRelation; @@ -37,6 +39,7 @@ import org.apache.doris.qe.StmtExecutor; import com.google.common.base.Preconditions; +import org.apache.commons.lang3.StringUtils; import java.util.List; import java.util.Map; @@ -51,12 +54,29 @@ public class AlterJobCommand extends AlterCommand implements ForwardWithSync, Ne private final String jobName; private final Map properties; private final String sql; + private final String sourceType; + private final String targetDb; + private final Map sourceProperties; + private final Map targetProperties; - public AlterJobCommand(String jobName, Map properties, String sql) { + /** + * AlterJobCommand constructor. + */ + public AlterJobCommand(String jobName, + Map properties, + String sql, + String sourceType, + String targetDb, + Map sourceProperties, + Map targetProperties) { super(PlanType.ALTER_JOB_COMMAND); this.jobName = jobName; this.properties = properties; this.sql = sql; + this.sourceType = sourceType; + this.targetDb = targetDb; + this.sourceProperties = sourceProperties; + this.targetProperties = targetProperties; } public String getJobName() { @@ -71,6 +91,22 @@ public String getSql() { return sql; } + public String getSourceType() { + return sourceType; + } + + public String getTargetDb() { + return targetDb; + } + + public Map getSourceProperties() { + return sourceProperties; + } + + public Map getTargetProperties() { + return targetProperties; + } + @Override public StmtType stmtType() { return StmtType.ALTER; @@ -100,26 +136,90 @@ private void validate() throws Exception { StreamingInsertJob streamingJob = (StreamingInsertJob) job; streamingJob.checkPrivilege(ConnectContext.get()); - boolean propModified = isPropertiesModified(streamingJob.getProperties()); - if (propModified) { - validateProps(streamingJob); - } - boolean sqlModified = isSqlModified(streamingJob.getExecuteSql()); - if (sqlModified) { - checkUnmodifiableProperties(streamingJob.getExecuteSql()); - } - if (!propModified && !sqlModified) { - throw new AnalysisException("No properties or sql changed in ALTER JOB"); + if (sourceType == null) { + boolean propModified = + isPropertiesModified(streamingJob.getProperties(), this.getProperties()); + if (propModified) { + validateProps(streamingJob); + } + boolean sqlModified = isSqlModified(streamingJob.getExecuteSql()); + if (sqlModified) { + checkUnmodifiableProperties(streamingJob.getExecuteSql()); + } + if (!propModified && !sqlModified) { + throw new AnalysisException("No properties or sql changed in ALTER JOB"); + } + } else { + if (!sourceType.toUpperCase().equals(streamingJob.getDataSourceType().name())) { + throw new AnalysisException("source type can't be modified in ALTER JOB"); + } + + if (StringUtils.isNotEmpty(targetDb) && !targetDb.equals(streamingJob.getTargetDb())) { + throw new AnalysisException("target database can't be modified in ALTER JOB"); + } + + boolean propModified = isPropertiesModified(streamingJob.getProperties(), this.getProperties()); + if (propModified) { + validateProps(streamingJob); + } + + boolean sourcePropModified = + isPropertiesModified(streamingJob.getSourceProperties(), this.getSourceProperties()); + if (sourcePropModified) { + DataSourceConfigValidator.validateSource(this.getSourceProperties()); + checkUnmodifiableSourceProperties(streamingJob.getSourceProperties()); + } + + boolean targetPropModified = + isPropertiesModified(streamingJob.getTargetProperties(), this.getTargetProperties()); + if (targetPropModified) { + DataSourceConfigValidator.validateTarget(this.getTargetProperties()); + } + if (!propModified && !targetPropModified && !sourcePropModified) { + throw new AnalysisException("No properties or source or target properties changed in ALTER JOB"); + } } } else { throw new AnalysisException("Unsupported job type for ALTER:" + job.getJobType()); } } + private void checkUnmodifiableSourceProperties(Map originSourceProperties) { + if (sourceProperties.containsKey(DataSourceConfigKeys.JDBC_URL)) { + Preconditions.checkArgument(Objects.equals( + originSourceProperties.get(DataSourceConfigKeys.JDBC_URL), + sourceProperties.get(DataSourceConfigKeys.JDBC_URL)), + "The jdbc_url property cannot be modified in ALTER JOB"); + } + + if (sourceProperties.containsKey(DataSourceConfigKeys.DATABASE)) { + Preconditions.checkArgument(Objects.equals( + originSourceProperties.get(DataSourceConfigKeys.DATABASE), + sourceProperties.get(DataSourceConfigKeys.DATABASE)), + "The database property cannot be modified in ALTER JOB"); + } + + if (sourceProperties.containsKey(DataSourceConfigKeys.INCLUDE_TABLES)) { + Preconditions.checkArgument(Objects.equals( + originSourceProperties.get(DataSourceConfigKeys.INCLUDE_TABLES), + sourceProperties.get(DataSourceConfigKeys.INCLUDE_TABLES)), + "The include_tables property cannot be modified in ALTER JOB"); + } + + if (sourceProperties.containsKey(DataSourceConfigKeys.EXCLUDE_TABLES)) { + Preconditions.checkArgument(Objects.equals( + originSourceProperties.get(DataSourceConfigKeys.EXCLUDE_TABLES), + sourceProperties.get(DataSourceConfigKeys.EXCLUDE_TABLES)), + "The exclude_tables property cannot be modified in ALTER JOB"); + } + } + private void validateProps(StreamingInsertJob streamingJob) throws AnalysisException { StreamingJobProperties jobProperties = new StreamingJobProperties(properties); jobProperties.validate(); - if (jobProperties.getOffsetProperty() != null) { + // from to job no need valiate offset in job properties + if (streamingJob.getDataSourceType() == null + && jobProperties.getOffsetProperty() != null) { streamingJob.validateOffset(jobProperties.getOffsetProperty()); } } @@ -165,11 +265,11 @@ private Pair, UnboundTVFRelation> getTargetTableAndTvf(String sql) return Pair.of(targetTable, unboundTVFRelation); } - private boolean isPropertiesModified(Map originProps) { - if (this.properties == null || this.properties.isEmpty()) { + private boolean isPropertiesModified(Map originProps, Map modifiedProps) { + if (modifiedProps == null || modifiedProps.isEmpty()) { return false; } - if (!Objects.equals(this.properties, originProps)) { + if (!Objects.equals(modifiedProps, originProps)) { return true; } return false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java index 03a423e1b0b850..1640143d277f01 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java @@ -22,12 +22,15 @@ import org.apache.doris.common.Config; import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.DataSourceConfigValidator; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.commands.info.CreateJobInfo; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; +import org.apache.commons.lang3.StringUtils; + /** * syntax: * CREATE @@ -69,6 +72,11 @@ private void validate() throws JobException { if (streamingJobCnt >= Config.max_streaming_job_num) { throw new JobException("Exceed max streaming job num limit in fe.conf:" + Config.max_streaming_job_num); } + + if (StringUtils.isNotEmpty(createJobInfo.getSourceType())) { + DataSourceConfigValidator.validateSource(createJobInfo.getSourceProperties()); + DataSourceConfigValidator.validateTarget(createJobInfo.getTargetProperties()); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java index 327f98d4c1ac8d..49749918d4c548 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java @@ -41,6 +41,7 @@ import org.apache.doris.qe.ConnectContext; import com.google.common.base.Strings; +import org.apache.commons.lang3.StringUtils; import java.util.Map; import java.util.Optional; @@ -72,7 +73,7 @@ public class CreateJobInfo { private final String executeSql; private final boolean streamingJob; private final Map jobProperties; - private final Optional sourceType; + private final String sourceType; private final String targetDb; private final Map sourceProperties; private final Map targetProperties; @@ -94,7 +95,7 @@ public CreateJobInfo(Optional labelNameOptional, Optional onceJo Optional intervalOptional, Optional intervalTimeUnitOptional, Optional startsTimeStampOptional, Optional endsTimeStampOptional, Optional immediateStartOptional, String comment, String executeSql, - boolean streamingJob, Map jobProperties, Optional sourceType, + boolean streamingJob, Map jobProperties, String sourceType, String targetDb, Map sourceProperties, Map targetProperties) { this.labelNameOptional = labelNameOptional; this.onceJobStartTimestampOptional = onceJobStartTimestampOptional; @@ -155,15 +156,15 @@ public AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws UserExcepti jobExecutionConfiguration.setTimerDefinition(timerDefinition); // set source type if (streamingJob) { - if (sourceType.isPresent()) { - DataSourceType dataSourceType = DataSourceType.valueOf(sourceType.get()); + if (StringUtils.isNotEmpty(sourceType)) { + DataSourceType dataSourceType = DataSourceType.valueOf(sourceType.toUpperCase()); return analyzeAndCreateFromSourceJob(dbName, jobExecutionConfiguration, jobProperties, targetDb, dataSourceType, sourceProperties, targetProperties); } else { return analyzeAndCreateStreamingInsertJob(executeSql, dbName, jobExecutionConfiguration, jobProperties); } } else { - if (sourceType.isPresent()) { + if (sourceType != null) { throw new AnalysisException("From..To Database is only supported in streaming job"); } return analyzeAndCreateInsertJob(executeSql, dbName, jobExecutionConfiguration); @@ -380,4 +381,20 @@ public static Long stripQuotesAndParseTimestamp(String str) { public boolean streamingJob() { return streamingJob; } + + public String getSourceType() { + return sourceType; + } + + public String getTargetDb() { + return targetDb; + } + + public Map getSourceProperties() { + return sourceProperties; + } + + public Map getTargetProperties() { + return targetProperties; + } } diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy index a5a4d47c9f5a6f..3613b81c3df735 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy @@ -122,10 +122,11 @@ suite("test_streaming_mysql_job", "p0,external,mysql,external_docker,external_do qt_select """ SELECT * FROM ${table1} order by name asc """ def jobInfo = sql """ - select loadStatistic from jobs("type"="insert") where Name='${jobName}' + select loadStatistic, status from jobs("type"="insert") where Name='${jobName}' """ log.info("jobInfo: " + jobInfo) assert jobInfo.get(0).get(0) == "{\"scannedRows\":7,\"loadBytes\":334,\"fileNumber\":0,\"fileSize\":0}" + assert jobInfo.get(0).get(1) == "RUNNING" sql """ DROP JOB IF EXISTS where jobname = '${jobName}' diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_create_alter.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_create_alter.groovy new file mode 100644 index 00000000000000..cfee6d89228b4e --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_create_alter.groovy @@ -0,0 +1,430 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job_create_alter", "p0,external,mysql,external_docker,external_docker_mysql") { + def jobName = "test_streaming_mysql_job_create_alter" + def currentDb = (sql "select database()")[0][0] + def table1 = "create_alter_user_info" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // unexcepted source properties + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial1" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Invalid value for key 'offset': initial1" + } + + // unexcepted target properties + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties1.replication_num" = "1" + ) + """ + exception "Only support target properties with prefix table.create.properties" + } + + //error jdbc url format + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc1:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Failed to parse db type from jdbcUrl" + } + + // error jdbc url format + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root12345", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Access denied for user" + } + + // no exist db or tables + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root12345", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Access denied for user" + } + + // no exist db or empty db + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "noexistdb", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "No tables found in database" + } + + // no match table + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "noexisttable", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Can not found match table in database" + } + + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('A1', 1);""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('B1', 2);""" + } + + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + // check job status and succeed task count larger than 1 + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + def jobInfo = sql """ + select status from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfo: " + jobInfo) + assert jobInfo.get(0).get(0) == "RUNNING" + + // alter job + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://xxx:xxx", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Only PAUSED job can be altered" + } + + sql "PAUSE JOB where jobname = '${jobName}'"; + def jobInfo2 = sql """ + select status from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfo: " + jobInfo2) + assert jobInfo2.get(0).get(0) == "PAUSED" + + // alter jdbc url + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://127.0.0.1:4456", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "The jdbc_url property cannot be modified in ALTER JOB" + } + + // alter database + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "updatedatabase", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "The database property cannot be modified in ALTER JOB" + } + + // alter include tables + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "changeTable", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "The include_tables property cannot be modified in ALTER JOB" + } + + // alter exclude tables + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "exclude_tables" = "xxxx", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "The exclude_tables property cannot be modified in ALTER JOB" + } + + // unexcept properties + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial", + "xxx"="xxx" + ) + TO DATABASE ${currentDb} + """ + exception "Unexpected key" + } + + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb}""" + + def jobInfoOrigin = sql """ + select CurrentOffset,LoadStatistic from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfoOrigin: " + jobInfoOrigin) + + // alter job properties + sql """ALTER JOB ${jobName} + PROPERTIES( + "max_interval" = "5" + ) """ + + sql "RESUME JOB where jobname = '${jobName}'"; + + + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobStatus = sql """ select status from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobStatus: " + jobStatus) + // check job status and succeed task count larger than 1 + jobStatus.size() == 1 && 'RUNNING' == jobStatus.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + def jobInfoCurrent = sql """ + select CurrentOffset,LoadStatistic,Properties,ExecuteSql from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfoCurrent: " + jobInfoCurrent) + assert jobInfoCurrent.get(0).get(0) == jobInfoOrigin.get(0).get(0) + assert jobInfoCurrent.get(0).get(1) == jobInfoOrigin.get(0).get(1) + assert jobInfoCurrent.get(0).get(2) == "{\"max_interval\":\"5\"}" + assert jobInfoCurrent.get(0).get(3).contains("latest") + + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } +} diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_restart_fe.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_restart_fe.groovy new file mode 100644 index 00000000000000..fbd798b601ee9e --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_restart_fe.groovy @@ -0,0 +1,118 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job_restart_fe", "docker,external,mysql,external_docker,external_docker_mysql") { + def jobName = "test_streaming_mysql_job_restart_fe" + def currentDb = (sql "select database()")[0][0] + def table1 = "restart_user_info" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // create test + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('A1', 1);""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('B1', 2);""" + } + + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + // check table created + def showTables = sql """ show tables from ${currentDb} like '${table1}'; """ + assert showTables.size() == 1 + + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + // check job status and succeed task count larger than 2 + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + def jobInfoBeforeRestart = sql """ + select loadStatistic, status, currentOffset from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfoBeforeRestart: " + jobInfoBeforeRestart) + assert jobInfoBeforeRestart.get(0).get(0) == "{\"scannedRows\":2,\"loadBytes\":94,\"fileNumber\":0,\"fileSize\":0}" + assert jobInfoBeforeRestart.get(0).get(1) == "RUNNING" + + // Restart FE + cluster.restartFrontends() + sleep(30000) + context.reconnectFe() + + // check is it consistent after restart + def jobAfterRestart = sql """ + select loadStatistic, status, currentOffset from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobAfterRestart: " + jobAfterRestart) + assert jobAfterRestart.get(0).get(0) == "{\"scannedRows\":2,\"loadBytes\":94,\"fileNumber\":0,\"fileSize\":0}" + assert jobAfterRestart.get(0).get(1) == "PAUSED" + assert jobAfterRestart.get(0).get(2) == jobInfoBeforeRestart.get(0).get(2) + + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } +} From 4c300299d2aa5f8be5dd95857bf3247691289cf7 Mon Sep 17 00:00:00 2001 From: JNSimba <676366545@qq.com> Date: Mon, 15 Dec 2025 18:10:58 +0800 Subject: [PATCH 27/27] fix restart case --- ...test_streaming_mysql_job_restart_fe.groovy | 178 +++++++++--------- 1 file changed, 93 insertions(+), 85 deletions(-) diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_restart_fe.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_restart_fe.groovy index fbd798b601ee9e..eb0e4866143ed4 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_restart_fe.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_restart_fe.groovy @@ -15,104 +15,112 @@ // specific language governing permissions and limitations // under the License. +import org.apache.doris.regression.suite.ClusterOptions import org.awaitility.Awaitility import static java.util.concurrent.TimeUnit.SECONDS -suite("test_streaming_mysql_job_restart_fe", "docker,external,mysql,external_docker,external_docker_mysql") { +suite("test_streaming_mysql_job_restart_fe", "docker,mysql,external_docker,external_docker_mysql") { def jobName = "test_streaming_mysql_job_restart_fe" - def currentDb = (sql "select database()")[0][0] - def table1 = "restart_user_info" - def mysqlDb = "test_cdc_db" + def options = new ClusterOptions() + options.setFeNum(1) + // run in cloud and not cloud + options.cloudMode = null - sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" - sql """drop table if exists ${currentDb}.${table1} force""" + docker(options) { + def currentDb = (sql "select database()")[0][0] + def table1 = "restart_user_info" + def mysqlDb = "test_cdc_db" - String enabled = context.config.otherConfigs.get("enableJdbcTest") - if (enabled != null && enabled.equalsIgnoreCase("true")) { - String mysql_port = context.config.otherConfigs.get("mysql_57_port"); - String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") - String s3_endpoint = getS3Endpoint() - String bucket = getS3BucketName() - String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" - // create test - connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { - sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" - sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" - sql """CREATE TABLE ${mysqlDb}.${table1} ( - `name` varchar(200) NOT NULL, - `age` int DEFAULT NULL, - PRIMARY KEY (`name`) - ) ENGINE=InnoDB""" - sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('A1', 1);""" - sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('B1', 2);""" - } + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" - sql """CREATE JOB ${jobName} - ON STREAMING - FROM MYSQL ( - "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", - "driver_url" = "${driver_url}", - "driver_class" = "com.mysql.cj.jdbc.Driver", - "user" = "root", - "password" = "123456", - "database" = "${mysqlDb}", - "include_tables" = "${table1}", - "offset" = "initial" - ) - TO DATABASE ${currentDb} ( - "table.create.properties.replication_num" = "1" - ) - """ - // check table created - def showTables = sql """ show tables from ${currentDb} like '${table1}'; """ - assert showTables.size() == 1 + // create test + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('A1', 1);""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('B1', 2);""" + } - // check job running - try { - Awaitility.await().atMost(300, SECONDS) - .pollInterval(1, SECONDS).until( - { - def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ - log.info("jobSuccendCount: " + jobSuccendCount) - // check job status and succeed task count larger than 2 - jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) - } - ) - } catch (Exception ex){ - def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" - def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" - log.info("show job: " + showjob) - log.info("show task: " + showtask) - throw ex; - } - - def jobInfoBeforeRestart = sql """ - select loadStatistic, status, currentOffset from jobs("type"="insert") where Name='${jobName}' - """ - log.info("jobInfoBeforeRestart: " + jobInfoBeforeRestart) - assert jobInfoBeforeRestart.get(0).get(0) == "{\"scannedRows\":2,\"loadBytes\":94,\"fileNumber\":0,\"fileSize\":0}" - assert jobInfoBeforeRestart.get(0).get(1) == "RUNNING" + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + // check table created + def showTables = sql """ show tables from ${currentDb} like '${table1}'; """ + assert showTables.size() == 1 - // Restart FE - cluster.restartFrontends() - sleep(30000) - context.reconnectFe() + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + // check job status and succeed task count larger than 2 + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } - // check is it consistent after restart - def jobAfterRestart = sql """ + def jobInfoBeforeRestart = sql """ select loadStatistic, status, currentOffset from jobs("type"="insert") where Name='${jobName}' - """ - log.info("jobAfterRestart: " + jobAfterRestart) - assert jobAfterRestart.get(0).get(0) == "{\"scannedRows\":2,\"loadBytes\":94,\"fileNumber\":0,\"fileSize\":0}" - assert jobAfterRestart.get(0).get(1) == "PAUSED" - assert jobAfterRestart.get(0).get(2) == jobInfoBeforeRestart.get(0).get(2) + """ + log.info("jobInfoBeforeRestart: " + jobInfoBeforeRestart) + assert jobInfoBeforeRestart.get(0).get(0) == "{\"scannedRows\":2,\"loadBytes\":94,\"fileNumber\":0,\"fileSize\":0}" + assert jobInfoBeforeRestart.get(0).get(1) == "RUNNING" + + // Restart FE + cluster.restartFrontends() + sleep(60000) + context.reconnectFe() + + // check is it consistent after restart + def jobAfterRestart = sql """ + select loadStatistic, status, currentOffset from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobAfterRestart: " + jobAfterRestart) + assert jobAfterRestart.get(0).get(0) == "{\"scannedRows\":2,\"loadBytes\":94,\"fileNumber\":0,\"fileSize\":0}" + assert jobAfterRestart.get(0).get(1) == "RUNNING" + assert jobAfterRestart.get(0).get(2) == jobInfoBeforeRestart.get(0).get(2) - sql """ - DROP JOB IF EXISTS where jobname = '${jobName}' - """ - def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" - assert jobCountRsp.get(0).get(0) == 0 + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } } }