From b392f2ac4ca6fb789dd2cda309b141cec30b8b17 Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Tue, 23 Jul 2024 17:34:55 +0800 Subject: [PATCH 01/11] [ISSUE #5040] Support gtid mode for sync data with mysql --- .../web/db/entity/EventMeshMysqlPosition.java | 6 + .../position/impl/MysqlPositionHandler.java | 8 +- .../src/main/resources/eventmesh.sql | 3 + .../mapper/EventMeshMysqlPositionMapper.xml | 28 +- .../connector/rdb/canal/CanalSinkConfig.java | 2 + .../rdb/canal/CanalSourceConfig.java | 4 + .../offset/canal/CanalRecordOffset.java | 5 + .../offset/canal/CanalRecordPartition.java | 2 + .../connector/canal/CanalConnectRecord.java | 6 + .../canal/dialect/AbstractDbDialect.java | 4 - .../connector/canal/dialect/DbDialect.java | 2 - .../connector/canal/dialect/MysqlDialect.java | 4 - .../SqlBuilderLoadInterceptor.java | 24 +- .../connector/canal/sink/DbLoadContext.java | 2 + .../connector/canal/sink/GtidBatch.java | 48 +++ .../canal/sink/GtidBatchManager.java | 45 ++ .../sink/connector/CanalSinkConnector.java | 403 +++++++++++++----- .../connector/canal/source/EntryParser.java | 33 +- .../connector/CanalSourceConnector.java | 49 ++- 19 files changed, 508 insertions(+), 170 deletions(-) create mode 100644 eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/GtidBatch.java create mode 100644 eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/GtidBatchManager.java diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshMysqlPosition.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshMysqlPosition.java index ffe3e446d4..65a38b54b5 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshMysqlPosition.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshMysqlPosition.java @@ -38,10 +38,16 @@ public class EventMeshMysqlPosition implements Serializable { private Integer jobID; + private String serverUUID; + private String address; private Long position; + private String gtid; + + private String currentGtid; + private Long timestamp; private String journalName; diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/impl/MysqlPositionHandler.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/impl/MysqlPositionHandler.java index 525fe02c0d..f2c174c3b7 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/impl/MysqlPositionHandler.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/impl/MysqlPositionHandler.java @@ -115,9 +115,12 @@ public boolean handler(ReportPositionRequest request, Metadata metadata) { CanalRecordOffset offset = (CanalRecordOffset) recordPosition.getRecordOffset(); if (offset != null) { position.setPosition(offset.getOffset()); + position.setGtid(offset.getGtid()); + position.setCurrentGtid(offset.getCurrentGtid()); } CanalRecordPartition partition = (CanalRecordPartition) recordPosition.getRecordPartition(); if (partition != null) { + position.setServerUUID(partition.getServerUUID()); position.setTimestamp(partition.getTimeStamp()); position.setJournalName(partition.getJournalName()); } @@ -148,13 +151,16 @@ public List handler(FetchPositionRequest request, Metadata metad request.getJobID())); List recordPositionList = new ArrayList<>(); for (EventMeshMysqlPosition position : positionList) { - RecordPosition recordPosition = new RecordPosition(); CanalRecordPartition partition = new CanalRecordPartition(); partition.setTimeStamp(position.getTimestamp()); partition.setJournalName(position.getJournalName()); + partition.setServerUUID(position.getServerUUID()); + RecordPosition recordPosition = new RecordPosition(); recordPosition.setRecordPartition(partition); CanalRecordOffset offset = new CanalRecordOffset(); offset.setOffset(position.getPosition()); + offset.setGtid(position.getGtid()); + offset.setCurrentGtid(position.getCurrentGtid()); recordPosition.setRecordOffset(offset); recordPositionList.add(recordPosition); } diff --git a/eventmesh-admin-server/src/main/resources/eventmesh.sql b/eventmesh-admin-server/src/main/resources/eventmesh.sql index 3b6fc9b777..226101661c 100644 --- a/eventmesh-admin-server/src/main/resources/eventmesh.sql +++ b/eventmesh-admin-server/src/main/resources/eventmesh.sql @@ -71,8 +71,11 @@ CREATE TABLE IF NOT EXISTS `event_mesh_job_info` ( CREATE TABLE IF NOT EXISTS `event_mesh_mysql_position` ( `id` int unsigned NOT NULL AUTO_INCREMENT, `jobID` int unsigned NOT NULL, + `serverUUID` varchar(50) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL, `address` varchar(50) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci NOT NULL, `position` bigint DEFAULT NULL, + `gtid` varchar(50) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL, + `currentGtid` varchar(50) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL, `timestamp` bigint DEFAULT NULL, `journalName` varchar(50) COLLATE utf8mb4_general_ci DEFAULT NULL, `createTime` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, diff --git a/eventmesh-admin-server/src/main/resources/mapper/EventMeshMysqlPositionMapper.xml b/eventmesh-admin-server/src/main/resources/mapper/EventMeshMysqlPositionMapper.xml index bc3a3292a2..cbb7c094d8 100644 --- a/eventmesh-admin-server/src/main/resources/mapper/EventMeshMysqlPositionMapper.xml +++ b/eventmesh-admin-server/src/main/resources/mapper/EventMeshMysqlPositionMapper.xml @@ -16,24 +16,28 @@ limitations under the License. --> + PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" + "http://mybatis.org/dtd/mybatis-3-mapper.dtd"> - - - - - - - - + + + + + + + + + + + - id,jobID,address, - position,timestamp,journalName, + id + ,jobID,serverUUID,address, + position,gtid,currentGtid,timestamp,journalName, createTime,updateTime diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkConfig.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkConfig.java index 85484b2ce9..80aec7bfe9 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkConfig.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkConfig.java @@ -39,6 +39,8 @@ public class CanalSinkConfig extends SinkConfig { // sync mode: field/row private SyncMode syncMode; + private boolean isGTIDMode = true; + // skip sink process exception private Boolean skipException = false; diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceConfig.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceConfig.java index d75ceb6b58..707f102901 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceConfig.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceConfig.java @@ -45,6 +45,10 @@ public class CanalSourceConfig extends SourceConfig { private Short clientId; + private String serverUUID; + + private boolean isGTIDMode = true; + private Integer batchSize = 10000; private Long batchTimeout = -1L; diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/offset/canal/CanalRecordOffset.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/offset/canal/CanalRecordOffset.java index 90c94c99bd..d0f2053f4d 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/offset/canal/CanalRecordOffset.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/offset/canal/CanalRecordOffset.java @@ -30,6 +30,11 @@ public class CanalRecordOffset extends RecordOffset { private Long offset; + // mysql instance gtid range + private String gtid; + + private String currentGtid; + @Override public Class getRecordOffsetClass() { return CanalRecordOffset.class; diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/offset/canal/CanalRecordPartition.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/offset/canal/CanalRecordPartition.java index 72d404bab9..ded82306e3 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/offset/canal/CanalRecordPartition.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/offset/canal/CanalRecordPartition.java @@ -29,6 +29,8 @@ @ToString public class CanalRecordPartition extends RecordPartition { + private String serverUUID; + private String journalName; private Long timeStamp; diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/CanalConnectRecord.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/CanalConnectRecord.java index a723b24dc3..36ecd158f6 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/CanalConnectRecord.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/CanalConnectRecord.java @@ -31,8 +31,14 @@ public class CanalConnectRecord { private String schemaName; + private String tableName; + // mysql instance gtid range + private String gtid; + + private String currentGtid; + /** * The business type of the changed data (I/U/D/C/A/E), consistent with the EventType defined in EntryProtocol in canal. */ diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/AbstractDbDialect.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/AbstractDbDialect.java index f5c2245b9f..4cf0f82ec9 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/AbstractDbDialect.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/AbstractDbDialect.java @@ -97,10 +97,6 @@ public SqlTemplate getSqlTemplate() { return sqlTemplate; } - public boolean isDRDS() { - return false; - } - public String getShardColumns(String schema, String table) { return null; } diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/DbDialect.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/DbDialect.java index a18edfd5b2..781c2fe954 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/DbDialect.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/DbDialect.java @@ -48,8 +48,6 @@ public interface DbDialect { public boolean isSupportMergeSql(); - public boolean isDRDS(); - public LobHandler getLobHandler(); public JdbcTemplate getJdbcTemplate(); diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/MysqlDialect.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/MysqlDialect.java index acd491ba64..bfe5628716 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/MysqlDialect.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/dialect/MysqlDialect.java @@ -50,10 +50,6 @@ public String getDefaultSchema() { return null; } - public boolean isDRDS() { - return false; - } - public String getDefaultCatalog() { return jdbcTemplate.queryForObject("select database()", String.class); } diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/interceptor/SqlBuilderLoadInterceptor.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/interceptor/SqlBuilderLoadInterceptor.java index 24d6b42f8b..0ad07577f9 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/interceptor/SqlBuilderLoadInterceptor.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/interceptor/SqlBuilderLoadInterceptor.java @@ -51,35 +51,21 @@ public boolean before(CanalSinkConfig sinkConfig, CanalConnectRecord record) { String shardColumns = null; if (type.isInsert()) { - if (CollectionUtils.isEmpty(record.getColumns()) - && (dbDialect.isDRDS())) { - // sql - sql = sqlTemplate.getInsertSql(schemaName, - record.getTableName(), - buildColumnNames(record.getKeys()), - buildColumnNames(record.getColumns())); - } else { - sql = sqlTemplate.getMergeSql(schemaName, + sql = sqlTemplate.getMergeSql(schemaName, record.getTableName(), buildColumnNames(record.getKeys()), buildColumnNames(record.getColumns()), new String[] {}, - !dbDialect.isDRDS(), + true, shardColumns); - } } else if (type.isUpdate()) { - boolean existOldKeys = !CollectionUtils.isEmpty(record.getOldKeys()); boolean rowMode = sinkConfig.getSyncMode().isRow(); String[] keyColumns = null; String[] otherColumns = null; if (existOldKeys) { keyColumns = buildColumnNames(record.getOldKeys()); - if (dbDialect.isDRDS()) { - otherColumns = buildColumnNames(record.getUpdatedColumns(), record.getUpdatedKeys()); - } else { - otherColumns = buildColumnNames(record.getUpdatedColumns(), record.getKeys()); - } + otherColumns = buildColumnNames(record.getUpdatedColumns(), record.getKeys()); } else { keyColumns = buildColumnNames(record.getKeys()); otherColumns = buildColumnNames(record.getUpdatedColumns()); @@ -91,10 +77,10 @@ public boolean before(CanalSinkConfig sinkConfig, CanalConnectRecord record) { keyColumns, otherColumns, new String[] {}, - !dbDialect.isDRDS(), + true, shardColumns); } else { - sql = sqlTemplate.getUpdateSql(schemaName, record.getTableName(), keyColumns, otherColumns, !dbDialect.isDRDS(), shardColumns); + sql = sqlTemplate.getUpdateSql(schemaName, record.getTableName(), keyColumns, otherColumns, true, shardColumns); } } else if (type.isDelete()) { sql = sqlTemplate.getDeleteSql(schemaName, diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/DbLoadContext.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/DbLoadContext.java index 561d894870..3498e87e7b 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/DbLoadContext.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/DbLoadContext.java @@ -28,6 +28,8 @@ @Data public class DbLoadContext { + private String gtid; + private List lastProcessedRecords; private List prepareRecords; diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/GtidBatch.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/GtidBatch.java new file mode 100644 index 0000000000..dd6559b832 --- /dev/null +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/GtidBatch.java @@ -0,0 +1,48 @@ +/* + * 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.eventmesh.connector.canal.sink; + +import org.apache.eventmesh.connector.canal.CanalConnectRecord; + +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; + +public class GtidBatch { + private int totalBatches; + private List> batches; + private int receivedBatchCount; + + public GtidBatch(int totalBatches) { + this.totalBatches = totalBatches; + this.batches = new CopyOnWriteArrayList<>(new List[totalBatches]); + this.receivedBatchCount = 0; + } + + public void addBatch(int batchIndex, List batchRecords) { + batches.set(batchIndex, batchRecords); + receivedBatchCount++; + } + + public List> getBatches() { + return batches; + } + + public boolean isComplete() { + return receivedBatchCount == totalBatches; + } +} diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/GtidBatchManager.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/GtidBatchManager.java new file mode 100644 index 0000000000..30060aa8f5 --- /dev/null +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/GtidBatchManager.java @@ -0,0 +1,45 @@ +/* + * 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.eventmesh.connector.canal.sink; + +import org.apache.eventmesh.connector.canal.CanalConnectRecord; + +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; + +public class GtidBatchManager { + + private static ConcurrentHashMap gtidBatchMap = new ConcurrentHashMap<>(); + + public static void addBatch(String gtid, int batchIndex, int totalBatches, List batchRecords) { + gtidBatchMap.computeIfAbsent(gtid, k -> new GtidBatch(totalBatches)).addBatch(batchIndex, batchRecords); + } + + public static GtidBatch getGtidBatch(String gtid) { + return gtidBatchMap.get(gtid); + } + + public static boolean isComplete(String gtid) { + GtidBatch batch = gtidBatchMap.get(gtid); + return batch != null && batch.isComplete(); + } + + public static void removeGtidBatch(String gtid) { + gtidBatchMap.remove(gtid); + } +} diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java index 1888e204ac..042f34a1a0 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java @@ -31,13 +31,14 @@ import org.apache.eventmesh.connector.canal.sink.DbLoadData; import org.apache.eventmesh.connector.canal.sink.DbLoadData.TableLoadData; import org.apache.eventmesh.connector.canal.sink.DbLoadMerger; +import org.apache.eventmesh.connector.canal.sink.GtidBatch; +import org.apache.eventmesh.connector.canal.sink.GtidBatchManager; import org.apache.eventmesh.openconnect.api.ConnectorCreateService; import org.apache.eventmesh.openconnect.api.connector.ConnectorContext; import org.apache.eventmesh.openconnect.api.connector.SinkConnectorContext; import org.apache.eventmesh.openconnect.api.sink.Sink; import org.apache.eventmesh.openconnect.offsetmgmt.api.data.ConnectRecord; - import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; @@ -51,6 +52,7 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -85,6 +87,8 @@ public class CanalSinkConnector implements Sink, ConnectorCreateService { private ExecutorService executor; + private ExecutorService gtidSingleExecutor; + private int batchSize = 50; private boolean useBatch = true; @@ -120,6 +124,7 @@ public void init(ConnectorContext connectorContext) throws Exception { new ArrayBlockingQueue<>(sinkConfig.getPoolSize() * 4), new NamedThreadFactory("canalSink"), new ThreadPoolExecutor.CallerRunsPolicy()); + gtidSingleExecutor = Executors.newSingleThreadExecutor(r -> new Thread(r, "gtidSingleExecutor")); } @Override @@ -140,6 +145,7 @@ public String name() { @Override public void stop() { executor.shutdown(); + gtidSingleExecutor.shutdown(); } @Override @@ -150,6 +156,8 @@ public void put(List sinkRecords) { canalConnectRecordList = filterRecord(canalConnectRecordList, sinkConfig); if (isDdlDatas(canalConnectRecordList)) { doDdl(context, canalConnectRecordList); + } else if (sinkConfig.isGTIDMode()) { + doLoadWithGtid(context, sinkConfig, connectRecord); } else { canalConnectRecordList = DbLoadMerger.merge(canalConnectRecordList); @@ -255,6 +263,57 @@ private void doLoad(DbLoadContext context, CanalSinkConfig sinkConfig, DbLoadDat batchDatas.clear(); } + private void doLoadWithGtid(DbLoadContext context, CanalSinkConfig sinkConfig, ConnectRecord connectRecord) { + int batchIndex = connectRecord.getExtension("batchIndex", Integer.class); + int totalBatches = connectRecord.getExtension("totalBatches", Integer.class); + List canalConnectRecordList = (List) connectRecord.getData(); + String gtid = canalConnectRecordList.get(0).getCurrentGtid(); + GtidBatchManager.addBatch(gtid, batchIndex, totalBatches, canalConnectRecordList); + // check whether the batch is complete + if (GtidBatchManager.isComplete(gtid)) { + GtidBatch batch = GtidBatchManager.getGtidBatch(gtid); + List> totalRows = batch.getBatches(); + List filteredRows = new ArrayList<>(); + for (List canalConnectRecords : totalRows) { + canalConnectRecords = filterRecord(canalConnectRecords, sinkConfig); + if (!CollectionUtils.isEmpty(canalConnectRecords)) { + for (final CanalConnectRecord record : canalConnectRecords) { + boolean filter = interceptor.before(sinkConfig, record); + filteredRows.add(record); + } + } + } + context.setGtid(gtid); + Future result = gtidSingleExecutor.submit(new DbLoadWorker(context, filteredRows, dbDialect, false, sinkConfig)); + Exception ex = null; + try { + ex = result.get(); + } catch (Exception e) { + ex = e; + } + Boolean skipException = sinkConfig.getSkipException(); + if (skipException != null && skipException) { + if (ex != null) { + // do skip + log.warn("skip exception for data : {} , caused by {}", + filteredRows, + ExceptionUtils.getFullStackTrace(ex)); + GtidBatchManager.removeGtidBatch(gtid); + } + } else { + if (ex != null) { + log.error("sink connector will shutdown by " + ex.getMessage(), ExceptionUtils.getFullStackTrace(ex)); + gtidSingleExecutor.shutdown(); + System.exit(1); + } else { + GtidBatchManager.removeGtidBatch(gtid); + } + } + } else { + log.info("Batch received, waiting for other batches."); + } + } + private List> split(List records) { List> result = new ArrayList<>(); if (records == null || records.isEmpty()) { @@ -294,12 +353,12 @@ private boolean canBatch(CanalConnectRecord source, CanalConnectRecord target) { } private void doTwoPhase(DbLoadContext context, CanalSinkConfig sinkConfig, List> totalRows, boolean canBatch) { - List> results = new ArrayList>(); + List> results = new ArrayList<>(); for (List rows : totalRows) { if (CollectionUtils.isEmpty(rows)) { continue; } - results.add(executor.submit(new DbLoadWorker(context, rows, dbDialect, canBatch))); + results.add(executor.submit(new DbLoadWorker(context, rows, dbDialect, canBatch, sinkConfig))); } boolean partFailed = false; @@ -328,7 +387,7 @@ private void doTwoPhase(DbLoadContext context, CanalSinkConfig sinkConfig, List< Boolean skipException = sinkConfig.getSkipException(); if (skipException != null && skipException) { for (CanalConnectRecord retryRecord : retryRecords) { - DbLoadWorker worker = new DbLoadWorker(context, Arrays.asList(retryRecord), dbDialect, false); + DbLoadWorker worker = new DbLoadWorker(context, Arrays.asList(retryRecord), dbDialect, false, sinkConfig); try { Exception ex = worker.call(); if (ex != null) { @@ -345,7 +404,7 @@ private void doTwoPhase(DbLoadContext context, CanalSinkConfig sinkConfig, List< } } } else { - DbLoadWorker worker = new DbLoadWorker(context, retryRecords, dbDialect, false); + DbLoadWorker worker = new DbLoadWorker(context, retryRecords, dbDialect, false, sinkConfig); try { Exception ex = worker.call(); if (ex != null) { @@ -353,7 +412,9 @@ private void doTwoPhase(DbLoadContext context, CanalSinkConfig sinkConfig, List< } } catch (Exception ex) { log.error("##load phase two failed!", ex); - throw new RuntimeException(ex); + log.error("sink connector will shutdown by " + ex.getMessage(), ex); + executor.shutdown(); + System.exit(1); } } } @@ -369,16 +430,21 @@ class DbLoadWorker implements Callable { private final DbDialect dbDialect; private final List records; private final boolean canBatch; + + private final CanalSinkConfig sinkConfig; + private final List allFailedRecords = new ArrayList<>(); private final List allProcessedRecords = new ArrayList<>(); private final List processedRecords = new ArrayList<>(); private final List failedRecords = new ArrayList<>(); - public DbLoadWorker(DbLoadContext context, List records, DbDialect dbDialect, boolean canBatch) { + public DbLoadWorker(DbLoadContext context, List records, DbDialect dbDialect, boolean canBatch, + CanalSinkConfig sinkConfig) { this.context = context; this.records = records; this.canBatch = canBatch; this.dbDialect = dbDialect; + this.sinkConfig = sinkConfig; } public Exception call() throws Exception { @@ -392,132 +458,239 @@ public Exception call() throws Exception { private Exception doCall() { RuntimeException error = null; ExecuteResult exeResult = null; - int index = 0; - while (index < records.size()) { - final List splitDatas = new ArrayList<>(); - if (useBatch && canBatch) { - int end = Math.min(index + batchSize, records.size()); - splitDatas.addAll(records.subList(index, end)); - index = end; - } else { - splitDatas.add(records.get(index)); - index = index + 1; - } + if (sinkConfig.isGTIDMode()) { int retryCount = 0; - while (true) { - try { - if (!CollectionUtils.isEmpty(failedRecords)) { - splitDatas.clear(); - splitDatas.addAll(failedRecords); - } else { - failedRecords.addAll(splitDatas); + final List toExecuteRecords = new ArrayList<>(); + try { + if (!CollectionUtils.isEmpty(failedRecords)) { + // if failedRecords not empty, make it retry + toExecuteRecords.addAll(failedRecords); + } else { + toExecuteRecords.addAll(records); + // add to failed record first, maybe get lob or datasource error + failedRecords.addAll(toExecuteRecords); + } + JdbcTemplate template = dbDialect.getJdbcTemplate(); + String sourceGtid = context.getGtid(); + if (StringUtils.isNotEmpty(sourceGtid)) { + String setGtid = "SET @@session.gtid_next = '" + sourceGtid + "';"; + template.execute(setGtid); + } else { + log.error("gtid is empty in gtid mode"); + throw new RuntimeException("gtid is empty in gtid mode"); + } + + final LobCreator lobCreator = dbDialect.getLobHandler().getLobCreator(); + int affect = (Integer) dbDialect.getTransactionTemplate().execute((TransactionCallback) status -> { + try { + failedRecords.clear(); + processedRecords.clear(); + int affect1 = 0; + for (CanalConnectRecord record : toExecuteRecords) { + int affects = template.update(record.getSql(), new PreparedStatementSetter() { + public void setValues(PreparedStatement ps) throws SQLException { + doPreparedStatement(ps, dbDialect, lobCreator, record); + } + }); + affect1 = affect1 + affects; + processStat(record, affects, false); + } + return affect1; + } catch (Exception e) { + // rollback + status.setRollbackOnly(); + throw new RuntimeException("Failed to executed", e); + } finally { + lobCreator.close(); } + }); + + // reset gtid + String resetGtid = "SET @@session.gtid_next = AUTOMATIC;"; + dbDialect.getJdbcTemplate().execute(resetGtid); + error = null; + exeResult = ExecuteResult.SUCCESS; + } catch (DeadlockLoserDataAccessException ex) { + error = new RuntimeException(ExceptionUtils.getFullStackTrace(ex)); + exeResult = ExecuteResult.RETRY; + } catch (Throwable ex) { + error = new RuntimeException(ExceptionUtils.getFullStackTrace(ex)); + exeResult = ExecuteResult.ERROR; + } - final LobCreator lobCreator = dbDialect.getLobHandler().getLobCreator(); - if (useBatch && canBatch) { - final String sql = splitDatas.get(0).getSql(); - int[] affects = new int[splitDatas.size()]; - affects = (int[]) dbDialect.getTransactionTemplate().execute((TransactionCallback) status -> { - try { - failedRecords.clear(); - processedRecords.clear(); - JdbcTemplate template = dbDialect.getJdbcTemplate(); - int[] affects1 = template.batchUpdate(sql, new BatchPreparedStatementSetter() { - - public void setValues(PreparedStatement ps, int idx) throws SQLException { - doPreparedStatement(ps, dbDialect, lobCreator, splitDatas.get(idx)); - } - - public int getBatchSize() { - return splitDatas.size(); - } - }); - return affects1; - } finally { - lobCreator.close(); - } - }); + if (ExecuteResult.SUCCESS == exeResult) { + allFailedRecords.addAll(failedRecords); + allProcessedRecords.addAll(processedRecords); + failedRecords.clear(); + processedRecords.clear(); + } else if (ExecuteResult.RETRY == exeResult) { + retryCount = retryCount + 1; + processedRecords.clear(); + failedRecords.clear(); + failedRecords.addAll(toExecuteRecords); + int retry = 3; + if (retryCount >= retry) { + processFailedDatas(toExecuteRecords.size()); + throw new RuntimeException(String.format("execute retry %s times failed", retryCount), error); + } else { + try { + int retryWait = 3000; + int wait = retryCount * retryWait; + wait = Math.max(wait, retryWait); + Thread.sleep(wait); + } catch (InterruptedException ex) { + Thread.interrupted(); + processFailedDatas(toExecuteRecords.size()); + throw new RuntimeException(ex); + } + } + } else { + processedRecords.clear(); + failedRecords.clear(); + failedRecords.addAll(toExecuteRecords); + processFailedDatas(toExecuteRecords.size()); + throw error; + } + } else { + int index = 0; + while (index < records.size()) { + final List toExecuteRecords = new ArrayList<>(); + if (useBatch && canBatch) { + int end = Math.min(index + batchSize, records.size()); + toExecuteRecords.addAll(records.subList(index, end)); + index = end; + } else { + toExecuteRecords.add(records.get(index)); + index = index + 1; + } - for (int i = 0; i < splitDatas.size(); i++) { - assert affects != null; - processStat(splitDatas.get(i), affects[i], true); + int retryCount = 0; + while (true) { + try { + if (!CollectionUtils.isEmpty(failedRecords)) { + toExecuteRecords.clear(); + toExecuteRecords.addAll(failedRecords); + } else { + failedRecords.addAll(toExecuteRecords); } - } else { - final CanalConnectRecord record = splitDatas.get(0); - int affect = 0; - affect = (Integer) dbDialect.getTransactionTemplate().execute((TransactionCallback) status -> { - try { - failedRecords.clear(); - processedRecords.clear(); - JdbcTemplate template = dbDialect.getJdbcTemplate(); - int affect1 = template.update(record.getSql(), new PreparedStatementSetter() { - - public void setValues(PreparedStatement ps) throws SQLException { - doPreparedStatement(ps, dbDialect, lobCreator, record); - } - }); - return affect1; - } finally { - lobCreator.close(); + + final LobCreator lobCreator = dbDialect.getLobHandler().getLobCreator(); + if (useBatch && canBatch) { + JdbcTemplate template = dbDialect.getJdbcTemplate(); + final String sql = toExecuteRecords.get(0).getSql(); + + int[] affects = new int[toExecuteRecords.size()]; + + affects = (int[]) dbDialect.getTransactionTemplate().execute((TransactionCallback) status -> { + try { + failedRecords.clear(); + processedRecords.clear(); + int[] affects1 = template.batchUpdate(sql, new BatchPreparedStatementSetter() { + + public void setValues(PreparedStatement ps, int idx) throws SQLException { + doPreparedStatement(ps, dbDialect, lobCreator, toExecuteRecords.get(idx)); + } + + public int getBatchSize() { + return toExecuteRecords.size(); + } + }); + return affects1; + } catch (Exception e) { + // rollback + status.setRollbackOnly(); + throw new RuntimeException("Failed to execute batch with GTID", e); + } finally { + lobCreator.close(); + } + }); + + for (int i = 0; i < toExecuteRecords.size(); i++) { + assert affects != null; + processStat(toExecuteRecords.get(i), affects[i], true); } - }); - processStat(record, affect, false); - } + } else { + final CanalConnectRecord record = toExecuteRecords.get(0); + JdbcTemplate template = dbDialect.getJdbcTemplate(); + int affect = 0; + affect = (Integer) dbDialect.getTransactionTemplate().execute((TransactionCallback) status -> { + try { + failedRecords.clear(); + processedRecords.clear(); + int affect1 = template.update(record.getSql(), new PreparedStatementSetter() { + + public void setValues(PreparedStatement ps) throws SQLException { + doPreparedStatement(ps, dbDialect, lobCreator, record); + } + }); + return affect1; + } catch (Exception e) { + // rollback + status.setRollbackOnly(); + throw new RuntimeException("Failed to executed", e); + } finally { + lobCreator.close(); + } + }); + processStat(record, affect, false); + } - error = null; - exeResult = ExecuteResult.SUCCESS; - } catch (DeadlockLoserDataAccessException ex) { - error = new RuntimeException(ExceptionUtils.getFullStackTrace(ex)); - exeResult = ExecuteResult.RETRY; - } catch (Throwable ex) { - error = new RuntimeException(ExceptionUtils.getFullStackTrace(ex)); - exeResult = ExecuteResult.ERROR; - } + error = null; + exeResult = ExecuteResult.SUCCESS; + } catch (DeadlockLoserDataAccessException ex) { + error = new RuntimeException(ExceptionUtils.getFullStackTrace(ex)); + exeResult = ExecuteResult.RETRY; + } catch (Throwable ex) { + error = new RuntimeException(ExceptionUtils.getFullStackTrace(ex)); + exeResult = ExecuteResult.ERROR; + } - if (ExecuteResult.SUCCESS == exeResult) { - allFailedRecords.addAll(failedRecords); - allProcessedRecords.addAll(processedRecords); - failedRecords.clear(); - processedRecords.clear(); - break; // do next eventData - } else if (ExecuteResult.RETRY == exeResult) { - retryCount = retryCount + 1; - processedRecords.clear(); - failedRecords.clear(); - failedRecords.addAll(splitDatas); - int retry = 3; - if (retryCount >= retry) { - processFailedDatas(index); - throw new RuntimeException(String.format("execute retry %s times failed", retryCount), error); - } else { - try { - int retryWait = 3000; - int wait = retryCount * retryWait; - wait = Math.max(wait, retryWait); - Thread.sleep(wait); - } catch (InterruptedException ex) { - Thread.interrupted(); + if (ExecuteResult.SUCCESS == exeResult) { + allFailedRecords.addAll(failedRecords); + allProcessedRecords.addAll(processedRecords); + failedRecords.clear(); + processedRecords.clear(); + break; // do next eventData + } else if (ExecuteResult.RETRY == exeResult) { + retryCount = retryCount + 1; + processedRecords.clear(); + failedRecords.clear(); + failedRecords.addAll(toExecuteRecords); + int retry = 3; + if (retryCount >= retry) { processFailedDatas(index); - throw new RuntimeException(ex); + throw new RuntimeException(String.format("execute retry %s times failed", retryCount), error); + } else { + try { + int retryWait = 3000; + int wait = retryCount * retryWait; + wait = Math.max(wait, retryWait); + Thread.sleep(wait); + } catch (InterruptedException ex) { + Thread.interrupted(); + processFailedDatas(index); + throw new RuntimeException(ex); + } } + } else { + processedRecords.clear(); + failedRecords.clear(); + failedRecords.addAll(toExecuteRecords); + processFailedDatas(index); + throw error; } - } else { - processedRecords.clear(); - failedRecords.clear(); - failedRecords.addAll(splitDatas); - processFailedDatas(index); - throw error; } } } + context.getFailedRecords().addAll(allFailedRecords); context.getProcessedRecords().addAll(allProcessedRecords); return null; } private void doPreparedStatement(PreparedStatement ps, DbDialect dbDialect, LobCreator lobCreator, - CanalConnectRecord record) throws SQLException { + CanalConnectRecord record) throws SQLException { EventType type = record.getEventType(); List columns = new ArrayList(); if (type.isInsert()) { @@ -528,11 +701,7 @@ private void doPreparedStatement(PreparedStatement ps, DbDialect dbDialect, LobC } else if (type.isUpdate()) { boolean existOldKeys = !CollectionUtils.isEmpty(record.getOldKeys()); columns.addAll(record.getUpdatedColumns()); - if (existOldKeys && dbDialect.isDRDS()) { - columns.addAll(record.getUpdatedKeys()); - } else { - columns.addAll(record.getKeys()); - } + columns.addAll(record.getKeys()); if (existOldKeys) { columns.addAll(record.getOldKeys()); } diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/EntryParser.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/EntryParser.java index 32c55ec42c..7261e5e1a2 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/EntryParser.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/EntryParser.java @@ -58,9 +58,17 @@ public Map> parse(CanalSourceConfig sourceConfig, switch (entry.getEntryType()) { case ROWDATA: RowChange rowChange = RowChange.parseFrom(entry.getStoreValue()); - needSync = checkNeedSync(sourceConfig, rowChange.getRowDatas(0)); - if (needSync) { - transactionDataBuffer.add(entry); + if (sourceConfig.getServerUUID() != null && sourceConfig.isGTIDMode()) { + String currentGtid = entry.getHeader().getPropsList().get(0).getValue(); + if (currentGtid.contains(sourceConfig.getServerUUID())) { + transactionDataBuffer.add(entry); + } + } else { + // if not gtid mode, need check weather the entry is loopback by specified column value + needSync = checkNeedSync(sourceConfig, rowChange.getRowDatas(0)); + if (needSync) { + transactionDataBuffer.add(entry); + } } break; case TRANSACTIONEND: @@ -163,6 +171,14 @@ private CanalConnectRecord internParse(CanalSourceConfig canalSourceConfig, Entr canalConnectRecord.setExecuteTime(entry.getHeader().getExecuteTime()); canalConnectRecord.setJournalName(entry.getHeader().getLogfileName()); canalConnectRecord.setBinLogOffset(entry.getHeader().getLogfileOffset()); + // if enabled gtid mode, gtid not null + if (canalSourceConfig.isGTIDMode()) { + String currentGtid = entry.getHeader().getPropsList().get(0).getValue(); + String gtidRange = replaceGtidRange(entry.getHeader().getGtid(), currentGtid, canalSourceConfig.getServerUUID()); + canalConnectRecord.setGtid(gtidRange); + canalConnectRecord.setCurrentGtid(currentGtid); + } + EventType eventType = canalConnectRecord.getEventType(); List beforeColumns = rowData.getBeforeColumnsList(); @@ -242,6 +258,17 @@ private CanalConnectRecord internParse(CanalSourceConfig canalSourceConfig, Entr return canalConnectRecord; } + public static String replaceGtidRange(String gtid, String currentGtid, String serverUUID) { + String[] gtidRangeArray = gtid.split(","); + for (int i = 0; i < gtidRangeArray.length; i++) { + String gtidRange = gtidRangeArray[i]; + if (gtidRange.startsWith(serverUUID)) { + gtidRangeArray[i] = gtidRange.replaceFirst("\\d+$", currentGtid.split(":")[1]); + } + } + return String.join(",", gtidRangeArray); + } + private void checkUpdateKeyColumns(Map oldKeyColumns, Map keyColumns) { if (oldKeyColumns.isEmpty()) { return; diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java index 577142e00c..d2a8d5d0b8 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java @@ -175,6 +175,9 @@ private Canal buildCanal(CanalSourceConfig sourceConfig) { parameter.setDbUsername(sourceConfig.getSourceConnectorConfig().getUserName()); parameter.setDbPassword(sourceConfig.getSourceConnectorConfig().getPassWord()); + // set if enabled gtid mode + parameter.setGtidEnable(sourceConfig.isGTIDMode()); + // check positions // example: Arrays.asList("{\"journalName\":\"mysql-bin.000001\",\"position\":6163L,\"timestamp\":1322803601000L}", // "{\"journalName\":\"mysql-bin.000001\",\"position\":6163L,\"timestamp\":1322803601000L}") @@ -188,6 +191,14 @@ private Canal buildCanal(CanalSourceConfig sourceConfig) { recordPositionMap.put("journalName", canalRecordPartition.getJournalName()); recordPositionMap.put("timestamp", canalRecordPartition.getTimeStamp()); recordPositionMap.put("position", canalRecordOffset.getOffset()); + String gtidRange = canalRecordOffset.getGtid(); + if (gtidRange != null) { + if (canalRecordOffset.getCurrentGtid() != null) { + gtidRange = EntryParser.replaceGtidRange(canalRecordOffset.getGtid(), canalRecordOffset.getCurrentGtid(), + sourceConfig.getServerUUID()); + } + recordPositionMap.put("gtid", gtidRange); + } positions.add(JsonUtils.toJSONString(recordPositionMap)); }); parameter.setPositions(positions); @@ -231,7 +242,13 @@ public void start() throws Exception { @Override public void commit(ConnectRecord record) { long batchId = Long.parseLong(record.getExtension("messageId")); - canalServer.ack(clientIdentity, batchId); + int batchIndex = record.getExtension("batchIndex", Integer.class); + int totalBatches = record.getExtension("totalBatches", Integer.class); + if (batchIndex == totalBatches - 1) { + log.debug("ack records batchIndex:{}, totalBatches:{}, batchId:{}", + batchIndex, totalBatches, batchId); + canalServer.ack(clientIdentity, batchId); + } } @Override @@ -297,21 +314,37 @@ public List poll() { if (!connectorRecordMap.isEmpty()) { Set>> entrySet = connectorRecordMap.entrySet(); for (Map.Entry> entry : entrySet) { - // Xid offset - Long binLogOffset = entry.getKey(); List connectRecordList = entry.getValue(); CanalConnectRecord lastRecord = entry.getValue().get(connectRecordList.size() - 1); CanalRecordPartition canalRecordPartition = new CanalRecordPartition(); + canalRecordPartition.setServerUUID(sourceConfig.getServerUUID()); canalRecordPartition.setJournalName(lastRecord.getJournalName()); canalRecordPartition.setTimeStamp(lastRecord.getExecuteTime()); - + // Xid offset with gtid + Long binLogOffset = entry.getKey(); CanalRecordOffset canalRecordOffset = new CanalRecordOffset(); canalRecordOffset.setOffset(binLogOffset); + if (StringUtils.isNotEmpty(lastRecord.getGtid()) && StringUtils.isNotEmpty(lastRecord.getCurrentGtid())) { + canalRecordOffset.setGtid(lastRecord.getGtid()); + canalRecordOffset.setCurrentGtid(lastRecord.getCurrentGtid()); + } - ConnectRecord connectRecord = new ConnectRecord(canalRecordPartition, canalRecordOffset, System.currentTimeMillis()); - connectRecord.addExtension("messageId", String.valueOf(message.getId())); - connectRecord.setData(connectRecordList); - result.add(connectRecord); + // split record list + List> splitLists = new ArrayList<>(); + for (int i = 0; i < connectRecordList.size(); i += sourceConfig.getBatchSize()) { + int end = Math.min(i + sourceConfig.getBatchSize(), connectRecordList.size()); + List subList = connectRecordList.subList(i, end); + splitLists.add(subList); + } + + for (int i = 0; i < splitLists.size(); i++) { + ConnectRecord connectRecord = new ConnectRecord(canalRecordPartition, canalRecordOffset, System.currentTimeMillis()); + connectRecord.addExtension("messageId", String.valueOf(message.getId())); + connectRecord.addExtension("batchIndex", i); + connectRecord.addExtension("totalBatches", splitLists.size()); + connectRecord.setData(splitLists.get(i)); + result.add(connectRecord); + } } } else { // for the message has been filtered need ack message From 95d4a008f146423806a5d75943054eae8e13dc2a Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Wed, 24 Jul 2024 11:16:17 +0800 Subject: [PATCH 02/11] fix conflicts with master --- .../connector/canal/sink/connector/CanalSinkConnector.java | 2 +- .../connector/canal/source/connector/CanalSourceConnector.java | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java index ff0c9d7946..b290d7f6af 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java @@ -278,7 +278,7 @@ private void doLoadWithGtid(DbLoadContext context, CanalSinkConfig sinkConfig, C List> totalRows = batch.getBatches(); List filteredRows = new ArrayList<>(); for (List canalConnectRecords : totalRows) { - canalConnectRecords = filterRecord(canalConnectRecords, sinkConfig); + canalConnectRecords = filterRecord(canalConnectRecords); if (!CollectionUtils.isEmpty(canalConnectRecords)) { for (final CanalConnectRecord record : canalConnectRecords) { boolean filter = interceptor.before(sinkConfig, record); diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java index bf2a8af27a..6cd575cb77 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java @@ -150,6 +150,8 @@ protected void startEventParserInternal(CanalEventParser parser, boolean isGroup return instance; } }); + DatabaseConnection.sourceConfig = sourceConfig.getSourceConnectorConfig(); + DatabaseConnection.initSourceConnection(); tableMgr = new RdbTableMgr(sourceConfig.getSourceConnectorConfig(), DatabaseConnection.sourceDataSource); } From 1c856428e60f491a3740161ea08b960ce7aa014f Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Wed, 24 Jul 2024 11:17:39 +0800 Subject: [PATCH 03/11] fix checkstyle error --- .../connector/canal/sink/connector/CanalSinkConnector.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java index b290d7f6af..5f3c0a2bca 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java @@ -31,9 +31,9 @@ import org.apache.eventmesh.connector.canal.sink.DbLoadData; import org.apache.eventmesh.connector.canal.sink.DbLoadData.TableLoadData; import org.apache.eventmesh.connector.canal.sink.DbLoadMerger; -import org.apache.eventmesh.connector.canal.source.table.RdbTableMgr; import org.apache.eventmesh.connector.canal.sink.GtidBatch; import org.apache.eventmesh.connector.canal.sink.GtidBatchManager; +import org.apache.eventmesh.connector.canal.source.table.RdbTableMgr; import org.apache.eventmesh.openconnect.api.ConnectorCreateService; import org.apache.eventmesh.openconnect.api.connector.ConnectorContext; import org.apache.eventmesh.openconnect.api.connector.SinkConnectorContext; From 036dbafe460af072c402ae16093afcf2eac3eaf3 Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Fri, 26 Jul 2024 10:24:25 +0800 Subject: [PATCH 04/11] [ISSUE #5044] Data synchronization strong verification in mariadb gtid mode --- .../connector/rdb/canal/CanalSinkConfig.java | 2 + .../rdb/canal/CanalSourceConfig.java | 2 + .../sink/connector/CanalSinkConnector.java | 17 ++++++--- .../connector/canal/source/EntryParser.java | 37 ++++++++++++------- .../connector/CanalSourceConnector.java | 15 +++++--- 5 files changed, 49 insertions(+), 24 deletions(-) diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkConfig.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkConfig.java index 80aec7bfe9..026f33f4fc 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkConfig.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkConfig.java @@ -41,6 +41,8 @@ public class CanalSinkConfig extends SinkConfig { private boolean isGTIDMode = true; + private boolean isMariaDB = true; + // skip sink process exception private Boolean skipException = false; diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceConfig.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceConfig.java index 707f102901..8331d32cb7 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceConfig.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceConfig.java @@ -47,6 +47,8 @@ public class CanalSourceConfig extends SourceConfig { private String serverUUID; + private boolean isMariaDB = true; + private boolean isGTIDMode = true; private Integer batchSize = 10000; diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java index 5f3c0a2bca..8ecda8e125 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java @@ -476,9 +476,11 @@ private Exception doCall() { } JdbcTemplate template = dbDialect.getJdbcTemplate(); String sourceGtid = context.getGtid(); - if (StringUtils.isNotEmpty(sourceGtid)) { - String setGtid = "SET @@session.gtid_next = '" + sourceGtid + "';"; - template.execute(setGtid); + if (StringUtils.isNotEmpty(sourceGtid) && !sinkConfig.isMariaDB()) { + String setMySQLGtid = "SET @@session.gtid_next = '" + sourceGtid + "';"; + template.execute(setMySQLGtid); + } else if (StringUtils.isNotEmpty(sourceGtid) && sinkConfig.isMariaDB()) { + throw new RuntimeException("unsupport gtid mode for mariaDB"); } else { log.error("gtid is empty in gtid mode"); throw new RuntimeException("gtid is empty in gtid mode"); @@ -510,8 +512,13 @@ public void setValues(PreparedStatement ps) throws SQLException { }); // reset gtid - String resetGtid = "SET @@session.gtid_next = AUTOMATIC;"; - dbDialect.getJdbcTemplate().execute(resetGtid); + if (sinkConfig.isMariaDB()) { + throw new RuntimeException("unsupport gtid mode for mariaDB"); + } else { + String resetMySQLGtid = "SET @@session.gtid_next = 'AUTOMATIC';"; + dbDialect.getJdbcTemplate().execute(resetMySQLGtid); + } + error = null; exeResult = ExecuteResult.SUCCESS; } catch (DeadlockLoserDataAccessException ex) { diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/EntryParser.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/EntryParser.java index 708d5d120c..5c4303588d 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/EntryParser.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/EntryParser.java @@ -49,7 +49,7 @@ public class EntryParser { public static Map> parse(CanalSourceConfig sourceConfig, List datas, - RdbTableMgr tables) { + RdbTableMgr tables) { List recordList = new ArrayList<>(); List transactionDataBuffer = new ArrayList<>(); // need check weather the entry is loopback @@ -60,9 +60,9 @@ public static Map> parse(CanalSourceConfig source switch (entry.getEntryType()) { case ROWDATA: RowChange rowChange = RowChange.parseFrom(entry.getStoreValue()); - if (sourceConfig.getServerUUID() != null && sourceConfig.isGTIDMode()) { - String currentGtid = entry.getHeader().getPropsList().get(0).getValue(); - if (currentGtid.contains(sourceConfig.getServerUUID())) { + // don't support gtid for mariadb + if (sourceConfig.getServerUUID() != null && sourceConfig.isGTIDMode() && !sourceConfig.isMariaDB()) { + if (checkGtidForEntry(entry, sourceConfig)) { transactionDataBuffer.add(entry); } } else { @@ -90,9 +90,14 @@ public static Map> parse(CanalSourceConfig source return recordMap; } + private static boolean checkGtidForEntry(Entry entry, CanalSourceConfig sourceConfig) { + String currentGtid = entry.getHeader().getPropsList().get(0).getValue(); + return currentGtid.contains(sourceConfig.getServerUUID()); + } + private static void parseRecordListWithEntryBuffer(CanalSourceConfig sourceConfig, - List recordList, - List transactionDataBuffer, RdbTableMgr tables) { + List recordList, + List transactionDataBuffer, RdbTableMgr tables) { for (Entry bufferEntry : transactionDataBuffer) { List recordParsedList = internParse(sourceConfig, bufferEntry, tables); if (CollectionUtils.isEmpty(recordParsedList)) { @@ -130,7 +135,7 @@ private static Column getColumnIgnoreCase(List columns, String columName } private static List internParse(CanalSourceConfig sourceConfig, Entry entry, - RdbTableMgr tableMgr) { + RdbTableMgr tableMgr) { String schemaName = entry.getHeader().getSchemaName(); String tableName = entry.getHeader().getTableName(); if (tableMgr.getTable(schemaName, tableName) == null) { @@ -169,7 +174,7 @@ private static List internParse(CanalSourceConfig sourceConf } private static CanalConnectRecord internParse(CanalSourceConfig canalSourceConfig, Entry entry, - RowChange rowChange, RowData rowData) { + RowChange rowChange, RowData rowData) { CanalConnectRecord canalConnectRecord = new CanalConnectRecord(); canalConnectRecord.setTableName(entry.getHeader().getTableName()); canalConnectRecord.setSchemaName(entry.getHeader().getSchemaName()); @@ -179,10 +184,16 @@ private static CanalConnectRecord internParse(CanalSourceConfig canalSourceConfi canalConnectRecord.setBinLogOffset(entry.getHeader().getLogfileOffset()); // if enabled gtid mode, gtid not null if (canalSourceConfig.isGTIDMode()) { - String currentGtid = entry.getHeader().getPropsList().get(0).getValue(); - String gtidRange = replaceGtidRange(entry.getHeader().getGtid(), currentGtid, canalSourceConfig.getServerUUID()); - canalConnectRecord.setGtid(gtidRange); - canalConnectRecord.setCurrentGtid(currentGtid); + if (canalSourceConfig.isMariaDB()) { + String currentGtid = entry.getHeader().getGtid(); + canalConnectRecord.setGtid(currentGtid); + canalConnectRecord.setCurrentGtid(currentGtid); + } else { + String currentGtid = entry.getHeader().getPropsList().get(0).getValue(); + String gtidRange = replaceGtidRange(entry.getHeader().getGtid(), currentGtid, canalSourceConfig.getServerUUID()); + canalConnectRecord.setGtid(gtidRange); + canalConnectRecord.setCurrentGtid(currentGtid); + } } EventType eventType = canalConnectRecord.getEventType(); @@ -276,7 +287,7 @@ public static String replaceGtidRange(String gtid, String currentGtid, String se } private static void checkUpdateKeyColumns(Map oldKeyColumns, - Map keyColumns) { + Map keyColumns) { if (oldKeyColumns.isEmpty()) { return; } diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java index 6cd575cb77..f0bbd09d3e 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java @@ -198,13 +198,16 @@ private Canal buildCanal(CanalSourceConfig sourceConfig) { recordPositionMap.put("journalName", canalRecordPartition.getJournalName()); recordPositionMap.put("timestamp", canalRecordPartition.getTimeStamp()); recordPositionMap.put("position", canalRecordOffset.getOffset()); - String gtidRange = canalRecordOffset.getGtid(); - if (gtidRange != null) { - if (canalRecordOffset.getCurrentGtid() != null) { - gtidRange = EntryParser.replaceGtidRange(canalRecordOffset.getGtid(), canalRecordOffset.getCurrentGtid(), - sourceConfig.getServerUUID()); + // for mariaDB not support gtid mode + if(sourceConfig.isGTIDMode() && !sourceConfig.isMariaDB()) { + String gtidRange = canalRecordOffset.getGtid(); + if (gtidRange != null) { + if (canalRecordOffset.getCurrentGtid() != null ) { + gtidRange = EntryParser.replaceGtidRange(canalRecordOffset.getGtid(), canalRecordOffset.getCurrentGtid(), + sourceConfig.getServerUUID()); + } + recordPositionMap.put("gtid", gtidRange); } - recordPositionMap.put("gtid", gtidRange); } positions.add(JsonUtils.toJSONString(recordPositionMap)); }); From 33bbbe4356b4a9987de65942eb7e438ca023c5bd Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Fri, 26 Jul 2024 10:55:36 +0800 Subject: [PATCH 05/11] fix checkstyle error --- .../canal/source/connector/CanalSourceConnector.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java index f0bbd09d3e..f3f8b2e160 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java @@ -199,10 +199,10 @@ private Canal buildCanal(CanalSourceConfig sourceConfig) { recordPositionMap.put("timestamp", canalRecordPartition.getTimeStamp()); recordPositionMap.put("position", canalRecordOffset.getOffset()); // for mariaDB not support gtid mode - if(sourceConfig.isGTIDMode() && !sourceConfig.isMariaDB()) { + if (sourceConfig.isGTIDMode() && !sourceConfig.isMariaDB()) { String gtidRange = canalRecordOffset.getGtid(); if (gtidRange != null) { - if (canalRecordOffset.getCurrentGtid() != null ) { + if (canalRecordOffset.getCurrentGtid() != null) { gtidRange = EntryParser.replaceGtidRange(canalRecordOffset.getGtid(), canalRecordOffset.getCurrentGtid(), sourceConfig.getServerUUID()); } From 2d45966e756c35d3d129bbca391bf29ad3bcb02a Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Mon, 29 Jul 2024 17:55:27 +0800 Subject: [PATCH 06/11] [ISSUE #5048] Add report verify request to admin for connector runtime --- .../common/enums/ConnectorStage.java | 23 ++++++++ .../remote/request/ReportVerifyRequest.java | 37 ++++++++++++ .../runtime/connector/ConnectorRuntime.java | 59 ++++++++++++++++++- .../connector/ConnectorRuntimeConfig.java | 6 ++ .../src/main/resources/connector.yaml | 2 + 5 files changed, 125 insertions(+), 2 deletions(-) create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/enums/ConnectorStage.java create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportVerifyRequest.java diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/enums/ConnectorStage.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/enums/ConnectorStage.java new file mode 100644 index 0000000000..90265fba4a --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/enums/ConnectorStage.java @@ -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. + */ + +package org.apache.eventmesh.common.enums; + +public enum ConnectorStage { + SOURCE, + SINK +} diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportVerifyRequest.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportVerifyRequest.java new file mode 100644 index 0000000000..f02b8aaf35 --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportVerifyRequest.java @@ -0,0 +1,37 @@ +/* + * 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.eventmesh.common.remote.request; + +import lombok.Data; +import lombok.EqualsAndHashCode; + +@Data +@EqualsAndHashCode(callSuper = true) +public class ReportVerifyRequest extends BaseRemoteRequest{ + private String taskID; + + private String recordID; + + private String recordSig; + + private String connectorName; + + private String connectorStage; + + private String position; +} diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java index 65676903dd..1605319862 100644 --- a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java @@ -25,6 +25,7 @@ import org.apache.eventmesh.common.config.connector.SinkConfig; import org.apache.eventmesh.common.config.connector.SourceConfig; import org.apache.eventmesh.common.config.connector.offset.OffsetStorageConfig; +import org.apache.eventmesh.common.enums.ConnectorStage; import org.apache.eventmesh.common.protocol.grpc.adminserver.AdminServiceGrpc; import org.apache.eventmesh.common.protocol.grpc.adminserver.AdminServiceGrpc.AdminServiceBlockingStub; import org.apache.eventmesh.common.protocol.grpc.adminserver.AdminServiceGrpc.AdminServiceStub; @@ -32,6 +33,7 @@ import org.apache.eventmesh.common.protocol.grpc.adminserver.Payload; import org.apache.eventmesh.common.remote.request.FetchJobRequest; import org.apache.eventmesh.common.remote.request.ReportHeartBeatRequest; +import org.apache.eventmesh.common.remote.request.ReportVerifyRequest; import org.apache.eventmesh.common.remote.response.FetchJobResponse; import org.apache.eventmesh.common.utils.IPUtils; import org.apache.eventmesh.common.utils.JsonUtils; @@ -55,10 +57,13 @@ import org.apache.commons.collections4.CollectionUtils; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; import java.util.ArrayList; import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -281,8 +286,9 @@ public void start() throws Exception { try { this.stop(); } catch (Exception ex) { - throw new RuntimeException(ex); + log.error("Failed to stop after exception", ex); } + throw new RuntimeException(e); } }); // start @@ -294,8 +300,9 @@ public void start() throws Exception { try { this.stop(); } catch (Exception ex) { - throw new RuntimeException(ex); + log.error("Failed to stop after exception", ex); } + throw new RuntimeException(e); } }); } @@ -304,6 +311,8 @@ public void start() throws Exception { public void stop() throws Exception { sourceConnector.stop(); sinkConnector.stop(); + sourceService.shutdown(); + sinkService.shutdown(); heartBeatExecutor.shutdown(); requestObserver.onCompleted(); if (channel != null && !channel.isShutdown()) { @@ -318,6 +327,11 @@ private void startSourceConnector() throws Exception { // TODO: use producer pub record to storage replace below if (connectorRecordList != null && !connectorRecordList.isEmpty()) { for (ConnectRecord record : connectorRecordList) { + // if enabled incremental data reporting consistency check + if (connectorRuntimeConfig.enableIncrementalDataConsistencyCheck) { + reportVerifyRequest(record, connectorRuntimeConfig, ConnectorStage.SOURCE); + } + queue.put(record); Optional submittedRecordPosition = prepareToUpdateRecordOffset(record); Optional callback = @@ -336,6 +350,43 @@ private void startSourceConnector() throws Exception { } } + private void reportVerifyRequest(ConnectRecord record, ConnectorRuntimeConfig connectorRuntimeConfig, ConnectorStage connectorStage) { + UUID uuid = UUID.randomUUID(); + String recordId = uuid.toString(); + String md5Str = md5(record.toString()); + ReportVerifyRequest reportVerifyRequest = new ReportVerifyRequest(); + reportVerifyRequest.setTaskID(connectorRuntimeConfig.getTaskID()); + reportVerifyRequest.setRecordID(recordId); + reportVerifyRequest.setRecordSig(md5Str); + reportVerifyRequest.setConnectorName( + IPUtils.getLocalAddress() + "_" + connectorRuntimeConfig.getJobID() + "_" + connectorRuntimeConfig.getRegion()); + reportVerifyRequest.setConnectorStage(connectorStage.name()); + reportVerifyRequest.setPosition(JsonUtils.toJSONString(record.getPosition())); + + Metadata metadata = Metadata.newBuilder().setType(ReportVerifyRequest.class.getSimpleName()).build(); + + Payload request = Payload.newBuilder().setMetadata(metadata) + .setBody(Any.newBuilder().setValue(UnsafeByteOperations.unsafeWrap(Objects.requireNonNull(JsonUtils.toJSONBytes(reportVerifyRequest)))) + .build()) + .build(); + + requestObserver.onNext(request); + } + + private String md5(String input) { + try { + MessageDigest md = MessageDigest.getInstance("MD5"); + byte[] messageDigest = md.digest(input.getBytes()); + StringBuilder sb = new StringBuilder(); + for (byte b : messageDigest) { + sb.append(String.format("%02x", b)); + } + return sb.toString(); + } catch (NoSuchAlgorithmException e) { + throw new RuntimeException(e); + } + } + public Optional prepareToUpdateRecordOffset(ConnectRecord record) { return Optional.of(this.offsetManagement.submitRecord(record.getPosition())); } @@ -426,6 +477,10 @@ private void startSinkConnector() throws Exception { List connectRecordList = new ArrayList<>(); connectRecordList.add(connectRecord); sinkConnector.put(connectRecordList); + // if enabled incremental data reporting consistency check + if (connectorRuntimeConfig.enableIncrementalDataConsistencyCheck) { + reportVerifyRequest(connectRecord, connectorRuntimeConfig, ConnectorStage.SINK); + } } } } diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntimeConfig.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntimeConfig.java index 901defc47d..5a58cce08e 100644 --- a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntimeConfig.java +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntimeConfig.java @@ -31,8 +31,12 @@ public class ConnectorRuntimeConfig { private String connectorRuntimeInstanceId; + private String taskID; + private String jobID; + private String region; + private String sourceConnectorType; private String sourceConnectorDesc; @@ -45,4 +49,6 @@ public class ConnectorRuntimeConfig { private Map sinkConnectorConfig; + public boolean enableIncrementalDataConsistencyCheck = true; + } diff --git a/eventmesh-runtime-v2/src/main/resources/connector.yaml b/eventmesh-runtime-v2/src/main/resources/connector.yaml index bc7bc20756..bf7f58028b 100644 --- a/eventmesh-runtime-v2/src/main/resources/connector.yaml +++ b/eventmesh-runtime-v2/src/main/resources/connector.yaml @@ -15,4 +15,6 @@ # limitations under the License. # +taskID: 1 jobID: 1 +region: region1 From 53b7b8cdd5e22276f8cd71b95568162c8f948d89 Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Mon, 29 Jul 2024 18:00:15 +0800 Subject: [PATCH 07/11] fix checkstyle error --- .../eventmesh/common/remote/request/ReportVerifyRequest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportVerifyRequest.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportVerifyRequest.java index f02b8aaf35..87f4581eb5 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportVerifyRequest.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportVerifyRequest.java @@ -22,7 +22,8 @@ @Data @EqualsAndHashCode(callSuper = true) -public class ReportVerifyRequest extends BaseRemoteRequest{ +public class ReportVerifyRequest extends BaseRemoteRequest { + private String taskID; private String recordID; From 03497689739212623a92e61535ee897821ec0224 Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Wed, 31 Jul 2024 21:07:40 +0800 Subject: [PATCH 08/11] [ISSUE #5052] Enhancement for source\sink connector --- .../sink/connector/CanalSinkConnector.java | 41 +++++++++++++- .../connector/CanalSinkFullConnector.java | 5 ++ .../connector/CanalSourceConnector.java | 5 ++ .../connector/CanalSourceFullConnector.java | 5 ++ .../connector/ChatGPTSourceConnector.java | 5 ++ .../sink/connector/DingDingSinkConnector.java | 5 ++ .../sink/connector/FileSinkConnector.java | 5 ++ .../source/connector/FileSourceConnector.java | 5 ++ .../http/sink/HttpSinkConnector.java | 5 ++ .../http/source/HttpSourceConnector.java | 5 ++ .../jdbc/sink/JdbcSinkConnector.java | 5 ++ .../jdbc/source/JdbcSourceConnector.java | 5 ++ .../sink/connector/KafkaSinkConnector.java | 5 ++ .../connector/KafkaSourceConnector.java | 5 ++ .../sink/connector/KnativeSinkConnector.java | 5 ++ .../connector/KnativeSourceConnector.java | 5 ++ .../sink/connector/LarkSinkConnector.java | 5 ++ .../sink/connector/MongodbSinkConnector.java | 5 ++ .../connector/MongodbSourceConnector.java | 5 ++ .../connector/OpenFunctionSinkConnector.java | 5 ++ .../OpenFunctionSourceConnector.java | 5 ++ .../sink/connector/PravegaSinkConnector.java | 5 ++ .../connector/PravegaSourceConnector.java | 5 ++ .../connector/PrometheusSourceConnector.java | 5 ++ .../sink/connector/PulsarSinkConnector.java | 5 ++ .../connector/PulsarSourceConnector.java | 5 ++ .../sink/connector/RabbitMQSinkConnector.java | 5 ++ .../connector/RabbitMQSourceConnector.java | 5 ++ .../sink/connector/RedisSinkConnector.java | 5 ++ .../connector/RedisSourceConnector.java | 5 ++ .../sink/connector/RocketMQSinkConnector.java | 5 ++ .../connector/RocketMQSourceConnector.java | 5 ++ .../source/connector/S3SourceConnector.java | 5 ++ .../source/MessageSendingOperations.java | 2 +- .../connector/SpringSourceConnector.java | 7 ++- .../spring/pub/SpringPubController.java | 10 ++-- .../eventmesh/openconnect/SourceWorker.java | 10 ++-- .../openconnect/api/connector/Connector.java | 11 +++- .../api/connector/SourceConnectorContext.java | 3 + .../api/callback/SendExceptionContext.java} | 6 +- .../api/callback/SendMessageCallback.java | 4 +- .../offsetmgmt}/api/callback/SendResult.java | 2 +- .../offsetmgmt/api/data/ConnectRecord.java | 25 ++++++++- .../runtime/connector/ConnectorRuntime.java | 55 +++++++++++++++---- .../connector/ConnectorRuntimeConfig.java | 2 + .../src/main/resources/connector.yaml | 1 + 46 files changed, 301 insertions(+), 38 deletions(-) rename eventmesh-openconnect/{eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/callback/SendExcepionContext.java => eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/callback/SendExceptionContext.java} (90%) rename eventmesh-openconnect/{eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect => eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt}/api/callback/SendMessageCallback.java (87%) rename eventmesh-openconnect/{eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect => eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt}/api/callback/SendResult.java (95%) diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java index 8ecda8e125..1792b74fbf 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java @@ -38,6 +38,8 @@ import org.apache.eventmesh.openconnect.api.connector.ConnectorContext; import org.apache.eventmesh.openconnect.api.connector.SinkConnectorContext; import org.apache.eventmesh.openconnect.api.sink.Sink; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendExceptionContext; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendResult; import org.apache.eventmesh.openconnect.offsetmgmt.api.data.ConnectRecord; import org.apache.commons.lang.StringUtils; @@ -146,6 +148,11 @@ public String name() { return this.sinkConfig.getSinkConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { executor.shutdown(); @@ -159,7 +166,7 @@ public void put(List sinkRecords) { List canalConnectRecordList = (List) connectRecord.getData(); canalConnectRecordList = filterRecord(canalConnectRecordList); if (isDdlDatas(canalConnectRecordList)) { - doDdl(context, canalConnectRecordList); + doDdl(context, canalConnectRecordList, connectRecord); } else if (sinkConfig.isGTIDMode()) { doLoadWithGtid(context, sinkConfig, connectRecord); } else { @@ -197,7 +204,7 @@ private List filterRecord(List canalConn .collect(Collectors.toList()); } - private void doDdl(DbLoadContext context, List canalConnectRecordList) { + private void doDdl(DbLoadContext context, List canalConnectRecordList, ConnectRecord connectRecord) { for (final CanalConnectRecord record : canalConnectRecordList) { try { Boolean result = jdbcTemplate.execute(new StatementCallback() { @@ -217,9 +224,30 @@ public Boolean doInStatement(Statement stmt) throws SQLException, DataAccessExce context.getFailedRecords().add(record); } } catch (Throwable e) { + connectRecord.getCallback().onException(buildSendExceptionContext(connectRecord, e)); throw new RuntimeException(e); } } + connectRecord.getCallback().onSuccess(convertToSendResult(connectRecord)); + } + + private SendExceptionContext buildSendExceptionContext(ConnectRecord record, Throwable e) { + SendExceptionContext sendExceptionContext = new SendExceptionContext(); + sendExceptionContext.setMessageId(record.getRecordId()); + sendExceptionContext.setCause(e); + if(org.apache.commons.lang3.StringUtils.isNotEmpty(record.getExtension("topic"))) { + sendExceptionContext.setTopic(record.getExtension("topic")); + } + return sendExceptionContext; + } + + private SendResult convertToSendResult(ConnectRecord record) { + SendResult result = new SendResult(); + result.setMessageId(record.getRecordId()); + if(org.apache.commons.lang3.StringUtils.isNotEmpty(record.getExtension("topic"))) { + result.setTopic(record.getExtension("topic")); + } + return result; } private void doBefore(List canalConnectRecordList, final DbLoadData loadData) { @@ -291,6 +319,9 @@ private void doLoadWithGtid(DbLoadContext context, CanalSinkConfig sinkConfig, C Exception ex = null; try { ex = result.get(); + if (ex == null) { + connectRecord.getCallback().onSuccess(convertToSendResult(connectRecord)); + } } catch (Exception e) { ex = e; } @@ -298,14 +329,16 @@ private void doLoadWithGtid(DbLoadContext context, CanalSinkConfig sinkConfig, C if (skipException != null && skipException) { if (ex != null) { // do skip - log.warn("skip exception for data : {} , caused by {}", + log.warn("skip exception will ack data : {} , caused by {}", filteredRows, ExceptionUtils.getFullStackTrace(ex)); GtidBatchManager.removeGtidBatch(gtid); + connectRecord.getCallback().onSuccess(convertToSendResult(connectRecord)); } } else { if (ex != null) { log.error("sink connector will shutdown by " + ex.getMessage(), ExceptionUtils.getFullStackTrace(ex)); + connectRecord.getCallback().onException(buildSendExceptionContext(connectRecord, ex)); gtidSingleExecutor.shutdown(); System.exit(1); } else { @@ -314,6 +347,8 @@ private void doLoadWithGtid(DbLoadContext context, CanalSinkConfig sinkConfig, C } } else { log.info("Batch received, waiting for other batches."); + // ack this record + connectRecord.getCallback().onSuccess(convertToSendResult(connectRecord)); } } diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkFullConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkFullConnector.java index 36c03b156c..2b4c9d7a94 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkFullConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkFullConnector.java @@ -109,6 +109,11 @@ public String name() { return null; } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void put(List sinkRecords) { if (sinkRecords == null || sinkRecords.isEmpty() || sinkRecords.get(0) == null) { diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java index f3f8b2e160..ea5ccdeed0 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceConnector.java @@ -267,6 +267,11 @@ public String name() { return this.sourceConfig.getSourceConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { if (!running) { diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceFullConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceFullConnector.java index df3c7571c2..97730463b5 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceFullConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceFullConnector.java @@ -159,6 +159,11 @@ public String name() { return this.config.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public List poll() { while (flag.get()) { diff --git a/eventmesh-connectors/eventmesh-connector-chatgpt/src/main/java/org/apache/eventmesh/connector/chatgpt/source/connector/ChatGPTSourceConnector.java b/eventmesh-connectors/eventmesh-connector-chatgpt/src/main/java/org/apache/eventmesh/connector/chatgpt/source/connector/ChatGPTSourceConnector.java index 4d54cb2191..6b122087e5 100644 --- a/eventmesh-connectors/eventmesh-connector-chatgpt/src/main/java/org/apache/eventmesh/connector/chatgpt/source/connector/ChatGPTSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-chatgpt/src/main/java/org/apache/eventmesh/connector/chatgpt/source/connector/ChatGPTSourceConnector.java @@ -224,6 +224,11 @@ public String name() { return this.sourceConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { Throwable t = this.server.close().cause(); diff --git a/eventmesh-connectors/eventmesh-connector-dingtalk/src/main/java/org/apache/eventmesh/connector/dingtalk/sink/connector/DingDingSinkConnector.java b/eventmesh-connectors/eventmesh-connector-dingtalk/src/main/java/org/apache/eventmesh/connector/dingtalk/sink/connector/DingDingSinkConnector.java index 417d9cef36..8c5a1e6611 100644 --- a/eventmesh-connectors/eventmesh-connector-dingtalk/src/main/java/org/apache/eventmesh/connector/dingtalk/sink/connector/DingDingSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-dingtalk/src/main/java/org/apache/eventmesh/connector/dingtalk/sink/connector/DingDingSinkConnector.java @@ -103,6 +103,11 @@ public String name() { return this.sinkConfig.getSinkConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { isRunning = false; diff --git a/eventmesh-connectors/eventmesh-connector-file/src/main/java/org/apache/eventmesh/connector/file/sink/connector/FileSinkConnector.java b/eventmesh-connectors/eventmesh-connector-file/src/main/java/org/apache/eventmesh/connector/file/sink/connector/FileSinkConnector.java index 89222b35b0..fabae0d43a 100644 --- a/eventmesh-connectors/eventmesh-connector-file/src/main/java/org/apache/eventmesh/connector/file/sink/connector/FileSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-file/src/main/java/org/apache/eventmesh/connector/file/sink/connector/FileSinkConnector.java @@ -103,6 +103,11 @@ public String name() { return this.sinkConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { outputStream.flush(); diff --git a/eventmesh-connectors/eventmesh-connector-file/src/main/java/org/apache/eventmesh/connector/file/source/connector/FileSourceConnector.java b/eventmesh-connectors/eventmesh-connector-file/src/main/java/org/apache/eventmesh/connector/file/source/connector/FileSourceConnector.java index 6ea0a0d33b..68b1a50989 100644 --- a/eventmesh-connectors/eventmesh-connector-file/src/main/java/org/apache/eventmesh/connector/file/source/connector/FileSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-file/src/main/java/org/apache/eventmesh/connector/file/source/connector/FileSourceConnector.java @@ -86,6 +86,11 @@ public String name() { return this.sourceConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { try { diff --git a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/HttpSinkConnector.java b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/HttpSinkConnector.java index 6d38b45306..8a14756372 100644 --- a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/HttpSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/HttpSinkConnector.java @@ -107,6 +107,11 @@ public String name() { return this.httpSinkConfig.connectorConfig.getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() throws Exception { this.sinkHandler.stop(); diff --git a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/source/HttpSourceConnector.java b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/source/HttpSourceConnector.java index 4155aff910..1ca325b18d 100644 --- a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/source/HttpSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/source/HttpSourceConnector.java @@ -144,6 +144,11 @@ public String name() { return this.sourceConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { if (this.server != null) { diff --git a/eventmesh-connectors/eventmesh-connector-jdbc/src/main/java/org/apache/eventmesh/connector/jdbc/sink/JdbcSinkConnector.java b/eventmesh-connectors/eventmesh-connector-jdbc/src/main/java/org/apache/eventmesh/connector/jdbc/sink/JdbcSinkConnector.java index 39681bf179..cc00f1e142 100644 --- a/eventmesh-connectors/eventmesh-connector-jdbc/src/main/java/org/apache/eventmesh/connector/jdbc/sink/JdbcSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-jdbc/src/main/java/org/apache/eventmesh/connector/jdbc/sink/JdbcSinkConnector.java @@ -139,6 +139,11 @@ public String name() { return this.sinkConfig.getSinkConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + /** * Stops the Connector. * diff --git a/eventmesh-connectors/eventmesh-connector-jdbc/src/main/java/org/apache/eventmesh/connector/jdbc/source/JdbcSourceConnector.java b/eventmesh-connectors/eventmesh-connector-jdbc/src/main/java/org/apache/eventmesh/connector/jdbc/source/JdbcSourceConnector.java index 2b2efcbef2..810a59e723 100644 --- a/eventmesh-connectors/eventmesh-connector-jdbc/src/main/java/org/apache/eventmesh/connector/jdbc/source/JdbcSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-jdbc/src/main/java/org/apache/eventmesh/connector/jdbc/source/JdbcSourceConnector.java @@ -192,6 +192,11 @@ public String name() { return "JDBC Source Connector"; } + @Override + public void onException(ConnectRecord record) { + + } + /** * Stops the Connector. * diff --git a/eventmesh-connectors/eventmesh-connector-kafka/src/main/java/org/apache/eventmesh/connector/kafka/sink/connector/KafkaSinkConnector.java b/eventmesh-connectors/eventmesh-connector-kafka/src/main/java/org/apache/eventmesh/connector/kafka/sink/connector/KafkaSinkConnector.java index b257cd0f44..0adafc1ce6 100644 --- a/eventmesh-connectors/eventmesh-connector-kafka/src/main/java/org/apache/eventmesh/connector/kafka/sink/connector/KafkaSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-kafka/src/main/java/org/apache/eventmesh/connector/kafka/sink/connector/KafkaSinkConnector.java @@ -94,6 +94,11 @@ public String name() { return this.sinkConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { producer.close(); diff --git a/eventmesh-connectors/eventmesh-connector-kafka/src/main/java/org/apache/eventmesh/connector/kafka/source/connector/KafkaSourceConnector.java b/eventmesh-connectors/eventmesh-connector-kafka/src/main/java/org/apache/eventmesh/connector/kafka/source/connector/KafkaSourceConnector.java index a3be1cbf93..d573126934 100644 --- a/eventmesh-connectors/eventmesh-connector-kafka/src/main/java/org/apache/eventmesh/connector/kafka/source/connector/KafkaSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-kafka/src/main/java/org/apache/eventmesh/connector/kafka/source/connector/KafkaSourceConnector.java @@ -94,6 +94,11 @@ public String name() { return this.sourceConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { kafkaConsumer.unsubscribe(); diff --git a/eventmesh-connectors/eventmesh-connector-knative/src/main/java/org/apache/eventmesh/connector/knative/sink/connector/KnativeSinkConnector.java b/eventmesh-connectors/eventmesh-connector-knative/src/main/java/org/apache/eventmesh/connector/knative/sink/connector/KnativeSinkConnector.java index a12a1c7461..b14f77ecd4 100644 --- a/eventmesh-connectors/eventmesh-connector-knative/src/main/java/org/apache/eventmesh/connector/knative/sink/connector/KnativeSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-knative/src/main/java/org/apache/eventmesh/connector/knative/sink/connector/KnativeSinkConnector.java @@ -82,6 +82,11 @@ public String name() { return this.sinkConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { started.compareAndSet(true, false); diff --git a/eventmesh-connectors/eventmesh-connector-knative/src/main/java/org/apache/eventmesh/connector/knative/source/connector/KnativeSourceConnector.java b/eventmesh-connectors/eventmesh-connector-knative/src/main/java/org/apache/eventmesh/connector/knative/source/connector/KnativeSourceConnector.java index 537c1ad4d9..1b0c033e8f 100644 --- a/eventmesh-connectors/eventmesh-connector-knative/src/main/java/org/apache/eventmesh/connector/knative/source/connector/KnativeSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-knative/src/main/java/org/apache/eventmesh/connector/knative/source/connector/KnativeSourceConnector.java @@ -65,6 +65,11 @@ public String name() { return this.sourceConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { started.compareAndSet(true, false); diff --git a/eventmesh-connectors/eventmesh-connector-lark/src/main/java/org/apache/eventmesh/connector/lark/sink/connector/LarkSinkConnector.java b/eventmesh-connectors/eventmesh-connector-lark/src/main/java/org/apache/eventmesh/connector/lark/sink/connector/LarkSinkConnector.java index d340dffd13..9981322e8f 100644 --- a/eventmesh-connectors/eventmesh-connector-lark/src/main/java/org/apache/eventmesh/connector/lark/sink/connector/LarkSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-lark/src/main/java/org/apache/eventmesh/connector/lark/sink/connector/LarkSinkConnector.java @@ -110,6 +110,11 @@ public String name() { return this.sinkConfig.getSinkConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { if (!started.compareAndSet(true, false)) { diff --git a/eventmesh-connectors/eventmesh-connector-mongodb/src/main/java/org/apache/eventmesh/connector/mongodb/sink/connector/MongodbSinkConnector.java b/eventmesh-connectors/eventmesh-connector-mongodb/src/main/java/org/apache/eventmesh/connector/mongodb/sink/connector/MongodbSinkConnector.java index 776ea8d71f..1001ffa584 100644 --- a/eventmesh-connectors/eventmesh-connector-mongodb/src/main/java/org/apache/eventmesh/connector/mongodb/sink/connector/MongodbSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-mongodb/src/main/java/org/apache/eventmesh/connector/mongodb/sink/connector/MongodbSinkConnector.java @@ -87,6 +87,11 @@ public String name() { return this.sinkConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() throws Exception { this.client.stop(); diff --git a/eventmesh-connectors/eventmesh-connector-mongodb/src/main/java/org/apache/eventmesh/connector/mongodb/source/connector/MongodbSourceConnector.java b/eventmesh-connectors/eventmesh-connector-mongodb/src/main/java/org/apache/eventmesh/connector/mongodb/source/connector/MongodbSourceConnector.java index e57c396719..df3f66d6a6 100644 --- a/eventmesh-connectors/eventmesh-connector-mongodb/src/main/java/org/apache/eventmesh/connector/mongodb/source/connector/MongodbSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-mongodb/src/main/java/org/apache/eventmesh/connector/mongodb/source/connector/MongodbSourceConnector.java @@ -93,6 +93,11 @@ public String name() { return this.sourceConfig.connectorConfig.getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() throws Exception { this.client.stop(); diff --git a/eventmesh-connectors/eventmesh-connector-openfunction/src/main/java/org/apache/eventmesh/connector/openfunction/sink/connector/OpenFunctionSinkConnector.java b/eventmesh-connectors/eventmesh-connector-openfunction/src/main/java/org/apache/eventmesh/connector/openfunction/sink/connector/OpenFunctionSinkConnector.java index 63444efe28..0f00a7e381 100644 --- a/eventmesh-connectors/eventmesh-connector-openfunction/src/main/java/org/apache/eventmesh/connector/openfunction/sink/connector/OpenFunctionSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-openfunction/src/main/java/org/apache/eventmesh/connector/openfunction/sink/connector/OpenFunctionSinkConnector.java @@ -74,6 +74,11 @@ public String name() { return this.sinkConfig.getSinkConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { } diff --git a/eventmesh-connectors/eventmesh-connector-openfunction/src/main/java/org/apache/eventmesh/connector/openfunction/source/connector/OpenFunctionSourceConnector.java b/eventmesh-connectors/eventmesh-connector-openfunction/src/main/java/org/apache/eventmesh/connector/openfunction/source/connector/OpenFunctionSourceConnector.java index b66bf9b18c..534ecfb79d 100644 --- a/eventmesh-connectors/eventmesh-connector-openfunction/src/main/java/org/apache/eventmesh/connector/openfunction/source/connector/OpenFunctionSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-openfunction/src/main/java/org/apache/eventmesh/connector/openfunction/source/connector/OpenFunctionSourceConnector.java @@ -76,6 +76,11 @@ public String name() { return this.sourceConfig.getSourceConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { diff --git a/eventmesh-connectors/eventmesh-connector-pravega/src/main/java/org/apache/eventmesh/connector/pravega/sink/connector/PravegaSinkConnector.java b/eventmesh-connectors/eventmesh-connector-pravega/src/main/java/org/apache/eventmesh/connector/pravega/sink/connector/PravegaSinkConnector.java index e5f09e4350..e089ef6760 100644 --- a/eventmesh-connectors/eventmesh-connector-pravega/src/main/java/org/apache/eventmesh/connector/pravega/sink/connector/PravegaSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-pravega/src/main/java/org/apache/eventmesh/connector/pravega/sink/connector/PravegaSinkConnector.java @@ -109,6 +109,11 @@ public String name() { return this.sinkConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { writerMap.forEach((topic, writer) -> writer.close()); diff --git a/eventmesh-connectors/eventmesh-connector-pravega/src/main/java/org/apache/eventmesh/connector/pravega/source/connector/PravegaSourceConnector.java b/eventmesh-connectors/eventmesh-connector-pravega/src/main/java/org/apache/eventmesh/connector/pravega/source/connector/PravegaSourceConnector.java index 2611617d8f..836779dbcf 100644 --- a/eventmesh-connectors/eventmesh-connector-pravega/src/main/java/org/apache/eventmesh/connector/pravega/source/connector/PravegaSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-pravega/src/main/java/org/apache/eventmesh/connector/pravega/source/connector/PravegaSourceConnector.java @@ -148,6 +148,11 @@ public String name() { return this.sourceConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { sourceHandlerMap.forEach((topic, handler) -> { diff --git a/eventmesh-connectors/eventmesh-connector-prometheus/src/main/java/org/apache/eventmesh/connector/prometheus/source/connector/PrometheusSourceConnector.java b/eventmesh-connectors/eventmesh-connector-prometheus/src/main/java/org/apache/eventmesh/connector/prometheus/source/connector/PrometheusSourceConnector.java index 5c78c718e3..0cafed73f3 100644 --- a/eventmesh-connectors/eventmesh-connector-prometheus/src/main/java/org/apache/eventmesh/connector/prometheus/source/connector/PrometheusSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-prometheus/src/main/java/org/apache/eventmesh/connector/prometheus/source/connector/PrometheusSourceConnector.java @@ -145,6 +145,11 @@ public String name() { return this.sourceConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { log.info("prometheus source connector stop."); diff --git a/eventmesh-connectors/eventmesh-connector-pulsar/src/main/java/org/apache/eventmesh/connector/pulsar/sink/connector/PulsarSinkConnector.java b/eventmesh-connectors/eventmesh-connector-pulsar/src/main/java/org/apache/eventmesh/connector/pulsar/sink/connector/PulsarSinkConnector.java index 9ff1f22a29..3f90c6c1be 100644 --- a/eventmesh-connectors/eventmesh-connector-pulsar/src/main/java/org/apache/eventmesh/connector/pulsar/sink/connector/PulsarSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-pulsar/src/main/java/org/apache/eventmesh/connector/pulsar/sink/connector/PulsarSinkConnector.java @@ -85,6 +85,11 @@ public String name() { return this.sinkConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { try { diff --git a/eventmesh-connectors/eventmesh-connector-pulsar/src/main/java/org/apache/eventmesh/connector/pulsar/source/connector/PulsarSourceConnector.java b/eventmesh-connectors/eventmesh-connector-pulsar/src/main/java/org/apache/eventmesh/connector/pulsar/source/connector/PulsarSourceConnector.java index 212d3eb487..0bc576221e 100644 --- a/eventmesh-connectors/eventmesh-connector-pulsar/src/main/java/org/apache/eventmesh/connector/pulsar/source/connector/PulsarSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-pulsar/src/main/java/org/apache/eventmesh/connector/pulsar/source/connector/PulsarSourceConnector.java @@ -87,6 +87,11 @@ public String name() { return this.sourceConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { try { diff --git a/eventmesh-connectors/eventmesh-connector-rabbitmq/src/main/java/org/apache/eventmesh/connector/rabbitmq/sink/connector/RabbitMQSinkConnector.java b/eventmesh-connectors/eventmesh-connector-rabbitmq/src/main/java/org/apache/eventmesh/connector/rabbitmq/sink/connector/RabbitMQSinkConnector.java index 4a94a2cb1f..08d1cefbac 100644 --- a/eventmesh-connectors/eventmesh-connector-rabbitmq/src/main/java/org/apache/eventmesh/connector/rabbitmq/sink/connector/RabbitMQSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-rabbitmq/src/main/java/org/apache/eventmesh/connector/rabbitmq/sink/connector/RabbitMQSinkConnector.java @@ -95,6 +95,11 @@ public String name() { return this.sinkConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { if (started) { diff --git a/eventmesh-connectors/eventmesh-connector-rabbitmq/src/main/java/org/apache/eventmesh/connector/rabbitmq/source/connector/RabbitMQSourceConnector.java b/eventmesh-connectors/eventmesh-connector-rabbitmq/src/main/java/org/apache/eventmesh/connector/rabbitmq/source/connector/RabbitMQSourceConnector.java index 655c20d9b9..0b7e726bda 100644 --- a/eventmesh-connectors/eventmesh-connector-rabbitmq/src/main/java/org/apache/eventmesh/connector/rabbitmq/source/connector/RabbitMQSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-rabbitmq/src/main/java/org/apache/eventmesh/connector/rabbitmq/source/connector/RabbitMQSourceConnector.java @@ -117,6 +117,11 @@ public String name() { return this.sourceConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { if (started) { diff --git a/eventmesh-connectors/eventmesh-connector-redis/src/main/java/org/apache/eventmesh/connector/redis/sink/connector/RedisSinkConnector.java b/eventmesh-connectors/eventmesh-connector-redis/src/main/java/org/apache/eventmesh/connector/redis/sink/connector/RedisSinkConnector.java index 83c3498a99..5b7d27c3ba 100644 --- a/eventmesh-connectors/eventmesh-connector-redis/src/main/java/org/apache/eventmesh/connector/redis/sink/connector/RedisSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-redis/src/main/java/org/apache/eventmesh/connector/redis/sink/connector/RedisSinkConnector.java @@ -85,6 +85,11 @@ public String name() { return this.sinkConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() throws Exception { this.redissonClient.shutdown(); diff --git a/eventmesh-connectors/eventmesh-connector-redis/src/main/java/org/apache/eventmesh/connector/redis/source/connector/RedisSourceConnector.java b/eventmesh-connectors/eventmesh-connector-redis/src/main/java/org/apache/eventmesh/connector/redis/source/connector/RedisSourceConnector.java index 70adce59e2..868639c205 100644 --- a/eventmesh-connectors/eventmesh-connector-redis/src/main/java/org/apache/eventmesh/connector/redis/source/connector/RedisSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-redis/src/main/java/org/apache/eventmesh/connector/redis/source/connector/RedisSourceConnector.java @@ -94,6 +94,11 @@ public String name() { return this.sourceConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() throws Exception { this.topic.removeAllListeners(); diff --git a/eventmesh-connectors/eventmesh-connector-rocketmq/src/main/java/org/apache/eventmesh/connector/rocketmq/sink/connector/RocketMQSinkConnector.java b/eventmesh-connectors/eventmesh-connector-rocketmq/src/main/java/org/apache/eventmesh/connector/rocketmq/sink/connector/RocketMQSinkConnector.java index ae9d4824e5..31d45a28f4 100644 --- a/eventmesh-connectors/eventmesh-connector-rocketmq/src/main/java/org/apache/eventmesh/connector/rocketmq/sink/connector/RocketMQSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-rocketmq/src/main/java/org/apache/eventmesh/connector/rocketmq/sink/connector/RocketMQSinkConnector.java @@ -78,6 +78,11 @@ public String name() { return this.sinkConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { producer.shutdown(); diff --git a/eventmesh-connectors/eventmesh-connector-rocketmq/src/main/java/org/apache/eventmesh/connector/rocketmq/source/connector/RocketMQSourceConnector.java b/eventmesh-connectors/eventmesh-connector-rocketmq/src/main/java/org/apache/eventmesh/connector/rocketmq/source/connector/RocketMQSourceConnector.java index 8ccb84acce..410f927d75 100644 --- a/eventmesh-connectors/eventmesh-connector-rocketmq/src/main/java/org/apache/eventmesh/connector/rocketmq/source/connector/RocketMQSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-rocketmq/src/main/java/org/apache/eventmesh/connector/rocketmq/source/connector/RocketMQSourceConnector.java @@ -206,6 +206,11 @@ public String name() { return this.sourceConfig.getConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { consumer.unsubscribe(sourceConfig.getConnectorConfig().getTopic()); diff --git a/eventmesh-connectors/eventmesh-connector-s3/src/main/java/org/apache/eventmesh/connector/s3/source/connector/S3SourceConnector.java b/eventmesh-connectors/eventmesh-connector-s3/src/main/java/org/apache/eventmesh/connector/s3/source/connector/S3SourceConnector.java index d0dc30c15e..078ed7691a 100644 --- a/eventmesh-connectors/eventmesh-connector-s3/src/main/java/org/apache/eventmesh/connector/s3/source/connector/S3SourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-s3/src/main/java/org/apache/eventmesh/connector/s3/source/connector/S3SourceConnector.java @@ -121,6 +121,11 @@ public String name() { return this.sourceConfig.getSourceConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() throws Exception { diff --git a/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/MessageSendingOperations.java b/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/MessageSendingOperations.java index a337c1cd81..5f38914bb1 100644 --- a/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/MessageSendingOperations.java +++ b/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/MessageSendingOperations.java @@ -17,7 +17,7 @@ package org.apache.eventmesh.connector.spring.source; -import org.apache.eventmesh.openconnect.api.callback.SendMessageCallback; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendMessageCallback; /** * Operations for sending messages. diff --git a/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/connector/SpringSourceConnector.java b/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/connector/SpringSourceConnector.java index 2ab5a3a3c0..be103a1f17 100644 --- a/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/connector/SpringSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/connector/SpringSourceConnector.java @@ -25,7 +25,7 @@ import org.apache.eventmesh.common.remote.offset.spring.SpringRecordPartition; import org.apache.eventmesh.connector.spring.source.MessageSendingOperations; import org.apache.eventmesh.openconnect.SourceWorker; -import org.apache.eventmesh.openconnect.api.callback.SendMessageCallback; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendMessageCallback; import org.apache.eventmesh.openconnect.api.connector.ConnectorContext; import org.apache.eventmesh.openconnect.api.connector.SourceConnectorContext; import org.apache.eventmesh.openconnect.api.source.Source; @@ -95,6 +95,11 @@ public String name() { return this.sourceConfig.getSourceConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() throws Exception { diff --git a/eventmesh-examples/src/main/java/org/apache/eventmesh/spring/pub/SpringPubController.java b/eventmesh-examples/src/main/java/org/apache/eventmesh/spring/pub/SpringPubController.java index b7ea8890ee..a734bb6efa 100644 --- a/eventmesh-examples/src/main/java/org/apache/eventmesh/spring/pub/SpringPubController.java +++ b/eventmesh-examples/src/main/java/org/apache/eventmesh/spring/pub/SpringPubController.java @@ -19,9 +19,9 @@ import org.apache.eventmesh.common.utils.JsonUtils; import org.apache.eventmesh.connector.spring.source.connector.SpringSourceConnector; -import org.apache.eventmesh.openconnect.api.callback.SendExcepionContext; -import org.apache.eventmesh.openconnect.api.callback.SendMessageCallback; -import org.apache.eventmesh.openconnect.api.callback.SendResult; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendExceptionContext; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendMessageCallback; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendResult; import java.util.HashMap; import java.util.Map; @@ -53,8 +53,8 @@ public void onSuccess(SendResult sendResult) { } @Override - public void onException(SendExcepionContext sendExcepionContext) { - log.info("Spring source worker send message to EventMesh failed!", sendExcepionContext.getCause()); + public void onException(SendExceptionContext sendExceptionContext) { + log.info("Spring source worker send message to EventMesh failed!", sendExceptionContext.getCause()); } }); return "success!"; diff --git a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/SourceWorker.java b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/SourceWorker.java index 6e48aa1de8..c3fa7e7cab 100644 --- a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/SourceWorker.java +++ b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/SourceWorker.java @@ -32,9 +32,9 @@ import org.apache.eventmesh.common.protocol.tcp.UserAgent; import org.apache.eventmesh.common.utils.JsonUtils; import org.apache.eventmesh.common.utils.SystemUtils; -import org.apache.eventmesh.openconnect.api.callback.SendExcepionContext; -import org.apache.eventmesh.openconnect.api.callback.SendMessageCallback; -import org.apache.eventmesh.openconnect.api.callback.SendResult; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendExceptionContext; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendMessageCallback; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendResult; import org.apache.eventmesh.openconnect.api.connector.SourceConnectorContext; import org.apache.eventmesh.openconnect.api.source.Source; import org.apache.eventmesh.openconnect.offsetmgmt.api.data.ConnectRecord; @@ -264,8 +264,8 @@ private SendResult convertToSendResult(CloudEvent event) { return result; } - private SendExcepionContext convertToExceptionContext(CloudEvent event, Throwable cause) { - SendExcepionContext exceptionContext = new SendExcepionContext(); + private SendExceptionContext convertToExceptionContext(CloudEvent event, Throwable cause) { + SendExceptionContext exceptionContext = new SendExceptionContext(); exceptionContext.setTopic(event.getId()); exceptionContext.setMessageId(event.getId()); exceptionContext.setCause(cause); diff --git a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/connector/Connector.java b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/connector/Connector.java index 8ac09eac38..07e44aea94 100644 --- a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/connector/Connector.java +++ b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/connector/Connector.java @@ -34,8 +34,7 @@ public interface Connector extends ComponentLifeCycle { Class configClass(); /** - * This init method is obsolete. For detailed discussion, - * please see here + * This init method is obsolete. For detailed discussion, please see here *

* Initializes the Connector with the provided configuration. * @@ -67,4 +66,12 @@ public interface Connector extends ComponentLifeCycle { */ String name(); + /** + * This method will be called when an exception occurs while processing a ConnectRecord object. This method can be used to handle the exception, + * such as logging error information, or stopping the connector's operation when an exception occurs. + * + * @param record The ConnectRecord object that was being processed when the exception occurred + */ + void onException(ConnectRecord record); + } diff --git a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/connector/SourceConnectorContext.java b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/connector/SourceConnectorContext.java index 55c88ce55a..f70e77248e 100644 --- a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/connector/SourceConnectorContext.java +++ b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/connector/SourceConnectorContext.java @@ -22,6 +22,7 @@ import org.apache.eventmesh.openconnect.offsetmgmt.api.storage.OffsetStorageReader; import java.util.List; +import java.util.Map; import lombok.Data; @@ -35,6 +36,8 @@ public class SourceConnectorContext implements ConnectorContext { public SourceConfig sourceConfig; + public Map runtimeConfig; + // initial record position public List recordPositionList; diff --git a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/callback/SendExcepionContext.java b/eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/callback/SendExceptionContext.java similarity index 90% rename from eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/callback/SendExcepionContext.java rename to eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/callback/SendExceptionContext.java index 0311ceaef5..974b19a547 100644 --- a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/callback/SendExcepionContext.java +++ b/eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/callback/SendExceptionContext.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package org.apache.eventmesh.openconnect.api.callback; +package org.apache.eventmesh.openconnect.offsetmgmt.api.callback; -public class SendExcepionContext { +public class SendExceptionContext { private String messageId; private String topic; private Throwable cause; - public SendExcepionContext() { + public SendExceptionContext() { } public String getMessageId() { diff --git a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/callback/SendMessageCallback.java b/eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/callback/SendMessageCallback.java similarity index 87% rename from eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/callback/SendMessageCallback.java rename to eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/callback/SendMessageCallback.java index fd6baba7ec..8346cf36b4 100644 --- a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/callback/SendMessageCallback.java +++ b/eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/callback/SendMessageCallback.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.eventmesh.openconnect.api.callback; +package org.apache.eventmesh.openconnect.offsetmgmt.api.callback; /** * Message sending callback interface. @@ -24,5 +24,5 @@ public interface SendMessageCallback { void onSuccess(SendResult sendResult); - void onException(SendExcepionContext sendExcepionContext); + void onException(SendExceptionContext sendExceptionContext); } diff --git a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/callback/SendResult.java b/eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/callback/SendResult.java similarity index 95% rename from eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/callback/SendResult.java rename to eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/callback/SendResult.java index 8cd861f6de..9afc745f3d 100644 --- a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/callback/SendResult.java +++ b/eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/callback/SendResult.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.eventmesh.openconnect.api.callback; +package org.apache.eventmesh.openconnect.offsetmgmt.api.callback; public class SendResult { diff --git a/eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/data/ConnectRecord.java b/eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/data/ConnectRecord.java index cda57e3758..b3fc4346c4 100644 --- a/eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/data/ConnectRecord.java +++ b/eventmesh-openconnect/eventmesh-openconnect-offsetmgmt-plugin/eventmesh-openconnect-offsetmgmt-api/src/main/java/org/apache/eventmesh/openconnect/offsetmgmt/api/data/ConnectRecord.java @@ -20,15 +20,19 @@ import org.apache.eventmesh.common.remote.offset.RecordOffset; import org.apache.eventmesh.common.remote.offset.RecordPartition; import org.apache.eventmesh.common.remote.offset.RecordPosition; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendMessageCallback; import java.util.Objects; import java.util.Set; +import java.util.UUID; /** * SourceDataEntries are generated by SourceTasks and passed to specific message queue to store. */ public class ConnectRecord { + private final String recordId = UUID.randomUUID().toString(); + private Long timestamp; private Object data; @@ -37,6 +41,8 @@ public class ConnectRecord { private KeyValue extensions; + private SendMessageCallback callback; + public ConnectRecord() { } @@ -57,6 +63,10 @@ public ConnectRecord(RecordPartition recordPartition, RecordOffset recordOffset, this.data = data; } + public String getRecordId() { + return recordId; + } + public Long getTimestamp() { return timestamp; } @@ -127,6 +137,14 @@ public Object getExtensionObj(String key) { return this.extensions.getObject(key); } + public SendMessageCallback getCallback() { + return callback; + } + + public void setCallback(SendMessageCallback callback) { + this.callback = callback; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -136,19 +154,20 @@ public boolean equals(Object o) { return false; } ConnectRecord that = (ConnectRecord) o; - return Objects.equals(timestamp, that.timestamp) && Objects.equals(data, that.data) + return Objects.equals(recordId, that.recordId) && Objects.equals(timestamp, that.timestamp) && Objects.equals(data, that.data) && Objects.equals(position, that.position) && Objects.equals(extensions, that.extensions); } @Override public int hashCode() { - return Objects.hash(timestamp, data, position, extensions); + return Objects.hash(recordId, timestamp, data, position, extensions); } @Override public String toString() { return "ConnectRecord{" - + "timestamp=" + timestamp + + "recordId=" + recordId + + ", timestamp=" + timestamp + ", data=" + data + ", position=" + position + ", extensions=" + extensions diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java index 1605319862..b13a5b35c5 100644 --- a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java @@ -38,7 +38,9 @@ import org.apache.eventmesh.common.utils.IPUtils; import org.apache.eventmesh.common.utils.JsonUtils; import org.apache.eventmesh.openconnect.api.ConnectorCreateService; -import org.apache.eventmesh.openconnect.api.callback.SendMessageCallback; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendExceptionContext; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendMessageCallback; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendResult; import org.apache.eventmesh.openconnect.api.connector.SinkConnectorContext; import org.apache.eventmesh.openconnect.api.connector.SourceConnectorContext; import org.apache.eventmesh.openconnect.api.factory.ConnectorPluginFactory; @@ -56,6 +58,7 @@ import org.apache.eventmesh.spi.EventMeshExtensionFactory; import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; @@ -207,6 +210,7 @@ private void initConnectorService() throws Exception { SourceConfig sourceConfig = (SourceConfig) ConfigUtil.parse(connectorRuntimeConfig.getSourceConnectorConfig(), sourceConnector.configClass()); SourceConnectorContext sourceConnectorContext = new SourceConnectorContext(); sourceConnectorContext.setSourceConfig(sourceConfig); + sourceConnectorContext.setRuntimeConfig(connectorRuntimeConfig.getRuntimeConfig()); sourceConnectorContext.setOffsetStorageReader(offsetStorageReader); if (CollectionUtils.isNotEmpty(jobResponse.getPosition())) { sourceConnectorContext.setRecordPositionList(jobResponse.getPosition()); @@ -332,15 +336,35 @@ private void startSourceConnector() throws Exception { reportVerifyRequest(record, connectorRuntimeConfig, ConnectorStage.SOURCE); } + // set a callback for this record + // if used the memory storage callback will be triggered after sink put success + record.setCallback(new SendMessageCallback() { + @Override + public void onSuccess(SendResult result) { + // commit record + sourceConnector.commit(record); + Optional submittedRecordPosition = prepareToUpdateRecordOffset(record); + submittedRecordPosition.ifPresent(RecordOffsetManagement.SubmittedPosition::ack); + Optional callback = + Optional.ofNullable(record.getExtensionObj(CALLBACK_EXTENSION)).map(v -> (SendMessageCallback) v); + callback.ifPresent(cb -> cb.onSuccess(convertToSendResult(record))); + } + + @Override + public void onException(SendExceptionContext sendExceptionContext) { + // handle exception + sourceConnector.onException(record); + log.error("send record to sink callback exception, process will shut down, record: {}", record, sendExceptionContext.getCause()); + try { + stop(); + } catch (Exception e) { + log.error("Failed to stop after exception", e); + } + } + }); + queue.put(record); - Optional submittedRecordPosition = prepareToUpdateRecordOffset(record); - Optional callback = - Optional.ofNullable(record.getExtensionObj(CALLBACK_EXTENSION)).map(v -> (SendMessageCallback) v); - // commit record - this.sourceConnector.commit(record); - submittedRecordPosition.ifPresent(RecordOffsetManagement.SubmittedPosition::ack); - // TODO:finish the optional callback - // callback.ifPresent(cb -> cb.onSuccess(record)); + offsetManagement.awaitAllMessages(5000, TimeUnit.MILLISECONDS); // update & commit offset updateCommittableOffsets(); @@ -350,13 +374,20 @@ private void startSourceConnector() throws Exception { } } + private SendResult convertToSendResult(ConnectRecord record) { + SendResult result = new SendResult(); + result.setMessageId(record.getRecordId()); + if(StringUtils.isNotEmpty(record.getExtension("topic"))) { + result.setTopic(record.getExtension("topic")); + } + return result; + } + private void reportVerifyRequest(ConnectRecord record, ConnectorRuntimeConfig connectorRuntimeConfig, ConnectorStage connectorStage) { - UUID uuid = UUID.randomUUID(); - String recordId = uuid.toString(); String md5Str = md5(record.toString()); ReportVerifyRequest reportVerifyRequest = new ReportVerifyRequest(); reportVerifyRequest.setTaskID(connectorRuntimeConfig.getTaskID()); - reportVerifyRequest.setRecordID(recordId); + reportVerifyRequest.setRecordID(record.getRecordId()); reportVerifyRequest.setRecordSig(md5Str); reportVerifyRequest.setConnectorName( IPUtils.getLocalAddress() + "_" + connectorRuntimeConfig.getJobID() + "_" + connectorRuntimeConfig.getRegion()); diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntimeConfig.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntimeConfig.java index 5a58cce08e..ab6fc3aaf5 100644 --- a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntimeConfig.java +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntimeConfig.java @@ -37,6 +37,8 @@ public class ConnectorRuntimeConfig { private String region; + private Map runtimeConfig; + private String sourceConnectorType; private String sourceConnectorDesc; diff --git a/eventmesh-runtime-v2/src/main/resources/connector.yaml b/eventmesh-runtime-v2/src/main/resources/connector.yaml index bf7f58028b..2e79e5cedc 100644 --- a/eventmesh-runtime-v2/src/main/resources/connector.yaml +++ b/eventmesh-runtime-v2/src/main/resources/connector.yaml @@ -18,3 +18,4 @@ taskID: 1 jobID: 1 region: region1 +runtimeConfig: # this used for connector runtime config From 0e76cbb69124302b2b7d6dfaaf3da46b929e92c0 Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Thu, 1 Aug 2024 12:40:12 +0800 Subject: [PATCH 09/11] fix checkstyle error --- .../canal/sink/connector/CanalSinkConnector.java | 4 ++-- .../slack/sink/connector/SlackSinkConnector.java | 5 +++++ .../spring/sink/connector/SpringSinkConnector.java | 5 +++++ .../spring/source/connector/SpringSourceConnector.java | 9 +++++---- .../wechat/sink/connector/WeChatSinkConnector.java | 5 +++++ .../wecom/sink/connector/WeComSinkConnector.java | 5 +++++ .../org/apache/eventmesh/openconnect/SourceWorker.java | 4 ++-- 7 files changed, 29 insertions(+), 8 deletions(-) diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java index 1792b74fbf..2ecb2384ac 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkConnector.java @@ -235,7 +235,7 @@ private SendExceptionContext buildSendExceptionContext(ConnectRecord record, Thr SendExceptionContext sendExceptionContext = new SendExceptionContext(); sendExceptionContext.setMessageId(record.getRecordId()); sendExceptionContext.setCause(e); - if(org.apache.commons.lang3.StringUtils.isNotEmpty(record.getExtension("topic"))) { + if (org.apache.commons.lang3.StringUtils.isNotEmpty(record.getExtension("topic"))) { sendExceptionContext.setTopic(record.getExtension("topic")); } return sendExceptionContext; @@ -244,7 +244,7 @@ private SendExceptionContext buildSendExceptionContext(ConnectRecord record, Thr private SendResult convertToSendResult(ConnectRecord record) { SendResult result = new SendResult(); result.setMessageId(record.getRecordId()); - if(org.apache.commons.lang3.StringUtils.isNotEmpty(record.getExtension("topic"))) { + if (org.apache.commons.lang3.StringUtils.isNotEmpty(record.getExtension("topic"))) { result.setTopic(record.getExtension("topic")); } return result; diff --git a/eventmesh-connectors/eventmesh-connector-slack/src/main/java/org/apache/eventmesh/connector/slack/sink/connector/SlackSinkConnector.java b/eventmesh-connectors/eventmesh-connector-slack/src/main/java/org/apache/eventmesh/connector/slack/sink/connector/SlackSinkConnector.java index e48760d506..836409af71 100644 --- a/eventmesh-connectors/eventmesh-connector-slack/src/main/java/org/apache/eventmesh/connector/slack/sink/connector/SlackSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-slack/src/main/java/org/apache/eventmesh/connector/slack/sink/connector/SlackSinkConnector.java @@ -84,6 +84,11 @@ public String name() { return this.sinkConfig.getSinkConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() { isRunning = false; diff --git a/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/sink/connector/SpringSinkConnector.java b/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/sink/connector/SpringSinkConnector.java index 94c40eea50..9ba99cd547 100644 --- a/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/sink/connector/SpringSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/sink/connector/SpringSinkConnector.java @@ -77,6 +77,11 @@ public String name() { return this.sinkConfig.getSinkConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() throws Exception { diff --git a/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/connector/SpringSourceConnector.java b/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/connector/SpringSourceConnector.java index be103a1f17..db286eb609 100644 --- a/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/connector/SpringSourceConnector.java +++ b/eventmesh-connectors/eventmesh-connector-spring/src/main/java/org/apache/eventmesh/connector/spring/source/connector/SpringSourceConnector.java @@ -25,10 +25,10 @@ import org.apache.eventmesh.common.remote.offset.spring.SpringRecordPartition; import org.apache.eventmesh.connector.spring.source.MessageSendingOperations; import org.apache.eventmesh.openconnect.SourceWorker; -import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendMessageCallback; import org.apache.eventmesh.openconnect.api.connector.ConnectorContext; import org.apache.eventmesh.openconnect.api.connector.SourceConnectorContext; import org.apache.eventmesh.openconnect.api.source.Source; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendMessageCallback; import org.apache.eventmesh.openconnect.offsetmgmt.api.data.ConnectRecord; import java.util.ArrayList; @@ -128,6 +128,7 @@ public List poll() { /** * Send message. + * * @param message message to send */ @Override @@ -141,9 +142,9 @@ public void send(Object message) { /** * Send message with a callback. - * @param message message to send. - * @param workerCallback After the user sends the message to the Connector, - * the SourceWorker will fetch message and invoke. + * + * @param message message to send. + * @param workerCallback After the user sends the message to the Connector, the SourceWorker will fetch message and invoke. */ @Override public void send(Object message, SendMessageCallback workerCallback) { diff --git a/eventmesh-connectors/eventmesh-connector-wechat/src/main/java/org/apache/eventmesh/connector/wechat/sink/connector/WeChatSinkConnector.java b/eventmesh-connectors/eventmesh-connector-wechat/src/main/java/org/apache/eventmesh/connector/wechat/sink/connector/WeChatSinkConnector.java index dec3f5e5de..6908d119b9 100644 --- a/eventmesh-connectors/eventmesh-connector-wechat/src/main/java/org/apache/eventmesh/connector/wechat/sink/connector/WeChatSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-wechat/src/main/java/org/apache/eventmesh/connector/wechat/sink/connector/WeChatSinkConnector.java @@ -115,6 +115,11 @@ public String name() { return this.sinkConfig.getSinkConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() throws IOException { isRunning = false; diff --git a/eventmesh-connectors/eventmesh-connector-wecom/src/main/java/org/apache/eventmesh/connector/wecom/sink/connector/WeComSinkConnector.java b/eventmesh-connectors/eventmesh-connector-wecom/src/main/java/org/apache/eventmesh/connector/wecom/sink/connector/WeComSinkConnector.java index ef6aed58c5..ca628fa590 100644 --- a/eventmesh-connectors/eventmesh-connector-wecom/src/main/java/org/apache/eventmesh/connector/wecom/sink/connector/WeComSinkConnector.java +++ b/eventmesh-connectors/eventmesh-connector-wecom/src/main/java/org/apache/eventmesh/connector/wecom/sink/connector/WeComSinkConnector.java @@ -95,6 +95,11 @@ public String name() { return this.sinkConfig.getSinkConnectorConfig().getConnectorName(); } + @Override + public void onException(ConnectRecord record) { + + } + @Override public void stop() throws IOException { isRunning = false; diff --git a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/SourceWorker.java b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/SourceWorker.java index c3fa7e7cab..2a2162a7af 100644 --- a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/SourceWorker.java +++ b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/SourceWorker.java @@ -32,11 +32,11 @@ import org.apache.eventmesh.common.protocol.tcp.UserAgent; import org.apache.eventmesh.common.utils.JsonUtils; import org.apache.eventmesh.common.utils.SystemUtils; +import org.apache.eventmesh.openconnect.api.connector.SourceConnectorContext; +import org.apache.eventmesh.openconnect.api.source.Source; import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendExceptionContext; import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendMessageCallback; import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendResult; -import org.apache.eventmesh.openconnect.api.connector.SourceConnectorContext; -import org.apache.eventmesh.openconnect.api.source.Source; import org.apache.eventmesh.openconnect.offsetmgmt.api.data.ConnectRecord; import org.apache.eventmesh.openconnect.offsetmgmt.api.data.RecordOffsetManagement; import org.apache.eventmesh.openconnect.offsetmgmt.api.storage.DefaultOffsetManagementServiceImpl; From c7706075c36157a1a685f9420c64dba14b4ed3bb Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Thu, 1 Aug 2024 12:46:27 +0800 Subject: [PATCH 10/11] fix checkstyle error --- .../runtime/connector/ConnectorRuntime.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java index dd248bf04d..6cd0452b83 100644 --- a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java @@ -38,14 +38,14 @@ import org.apache.eventmesh.common.utils.IPUtils; import org.apache.eventmesh.common.utils.JsonUtils; import org.apache.eventmesh.openconnect.api.ConnectorCreateService; -import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendExceptionContext; -import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendMessageCallback; -import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendResult; import org.apache.eventmesh.openconnect.api.connector.SinkConnectorContext; import org.apache.eventmesh.openconnect.api.connector.SourceConnectorContext; import org.apache.eventmesh.openconnect.api.factory.ConnectorPluginFactory; import org.apache.eventmesh.openconnect.api.sink.Sink; import org.apache.eventmesh.openconnect.api.source.Source; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendExceptionContext; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendMessageCallback; +import org.apache.eventmesh.openconnect.offsetmgmt.api.callback.SendResult; import org.apache.eventmesh.openconnect.offsetmgmt.api.data.ConnectRecord; import org.apache.eventmesh.openconnect.offsetmgmt.api.data.RecordOffsetManagement; import org.apache.eventmesh.openconnect.offsetmgmt.api.storage.DefaultOffsetManagementServiceImpl; @@ -66,7 +66,6 @@ import java.util.List; import java.util.Objects; import java.util.Optional; -import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -354,7 +353,8 @@ public void onSuccess(SendResult result) { public void onException(SendExceptionContext sendExceptionContext) { // handle exception sourceConnector.onException(record); - log.error("send record to sink callback exception, process will shut down, record: {}", record, sendExceptionContext.getCause()); + log.error("send record to sink callback exception, process will shut down, record: {}", record, + sendExceptionContext.getCause()); try { stop(); } catch (Exception e) { @@ -377,7 +377,7 @@ public void onException(SendExceptionContext sendExceptionContext) { private SendResult convertToSendResult(ConnectRecord record) { SendResult result = new SendResult(); result.setMessageId(record.getRecordId()); - if(StringUtils.isNotEmpty(record.getExtension("topic"))) { + if (StringUtils.isNotEmpty(record.getExtension("topic"))) { result.setTopic(record.getExtension("topic")); } return result; From 7af3d1609919963732d33788f119440a6fae9c49 Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Fri, 2 Aug 2024 20:48:53 +0800 Subject: [PATCH 11/11] [ISSUE #5067] Enhancement for eventmesh-admin-server --- eventmesh-admin-server/conf/application.yaml | 10 ++- eventmesh-admin-server/conf/eventmesh.sql | 14 +-- .../conf/mapper/EventMeshJobInfoMapper.xml | 40 ++++----- .../conf/mapper/EventMeshTaskInfoMapper.xml | 13 +-- .../admin/server/AdminServerProperties.java | 5 ++ .../admin/server/web/HttpServer.java | 9 +- .../web/db/entity/EventMeshJobInfo.java | 8 +- .../web/db/entity/EventMeshTaskInfo.java | 10 ++- .../db/mapper/EventMeshJobInfoExtMapper.java | 18 +++- .../impl/EventMeshVerifyServiceImpl.java | 39 +++++++++ .../admin/server/web/pojo/JobDetail.java | 8 +- .../web/service/job/JobInfoBizService.java | 30 ++++--- .../web/service/task/TaskBizService.java | 55 +++++++++--- .../common/remote/TransportType.java | 6 +- .../common/remote/datasource/DataSource.java | 25 +++--- .../MySqlIncDataSourceSourceConf.java | 85 ------------------- .../request/CreateOrUpdateDataSourceReq.java | 5 +- .../remote/request/CreateTaskRequest.java | 33 ++++++- .../runtime/RuntimeInstanceConfig.java | 4 +- .../runtime/boot/RuntimeInstance.java | 44 +++++----- .../runtime/connector/ConnectorRuntime.java | 2 +- .../src/main/resources/runtime.yaml | 2 + 22 files changed, 272 insertions(+), 193 deletions(-) create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshVerifyServiceImpl.java delete mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/MySqlIncDataSourceSourceConf.java diff --git a/eventmesh-admin-server/conf/application.yaml b/eventmesh-admin-server/conf/application.yaml index 54795057cb..afbcd4a438 100644 --- a/eventmesh-admin-server/conf/application.yaml +++ b/eventmesh-admin-server/conf/application.yaml @@ -28,5 +28,11 @@ mybatis-plus: log-impl: org.apache.ibatis.logging.stdout.StdOutImpl event-mesh: admin-server: - service-name: DEFAULT_GROUP@@em_adm_server - port: 8081 \ No newline at end of file + serviceName: DEFAULT_GROUP@@em_adm_server + port: 8081 + adminServerList: + region1: + - http://localhost:8081 + region2: + - http://localhost:8082 + region: region1 \ No newline at end of file diff --git a/eventmesh-admin-server/conf/eventmesh.sql b/eventmesh-admin-server/conf/eventmesh.sql index 82d5c53317..94edbb6fac 100644 --- a/eventmesh-admin-server/conf/eventmesh.sql +++ b/eventmesh-admin-server/conf/eventmesh.sql @@ -45,14 +45,15 @@ CREATE TABLE IF NOT EXISTS `event_mesh_data_source` ( CREATE TABLE IF NOT EXISTS `event_mesh_job_info` ( `id` int unsigned NOT NULL AUTO_INCREMENT, `jobID` varchar(50) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci NOT NULL, - `desc` varchar(50) COLLATE utf8mb4_general_ci NOT NULL, + `jobDesc` varchar(50) COLLATE utf8mb4_general_ci NOT NULL, `taskID` varchar(50) COLLATE utf8mb4_general_ci NOT NULL DEFAULT '', `transportType` varchar(50) COLLATE utf8mb4_general_ci NOT NULL DEFAULT '', `sourceData` int NOT NULL DEFAULT '0', `targetData` int NOT NULL DEFAULT '0', - `state` varchar(50) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci NOT NULL DEFAULT '', + `jobState` varchar(50) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci NOT NULL DEFAULT '', `jobType` varchar(50) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci NOT NULL DEFAULT '', `fromRegion` varchar(50) COLLATE utf8mb4_general_ci DEFAULT NULL, + `runningRegion` varchar(50) COLLATE utf8mb4_general_ci DEFAULT NULL, `createUid` varchar(50) COLLATE utf8mb4_general_ci DEFAULT NULL, `updateUid` varchar(50) COLLATE utf8mb4_general_ci DEFAULT NULL, `createTime` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, @@ -118,10 +119,11 @@ CREATE TABLE IF NOT EXISTS `event_mesh_runtime_history` ( CREATE TABLE IF NOT EXISTS `event_mesh_task_info` ( `id` int unsigned NOT NULL AUTO_INCREMENT, `taskID` varchar(50) COLLATE utf8mb4_general_ci NOT NULL, - `name` varchar(50) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci NOT NULL, - `desc` varchar(50) COLLATE utf8mb4_general_ci NOT NULL, - `state` varchar(50) COLLATE utf8mb4_general_ci NOT NULL DEFAULT '' COMMENT 'taskstate', - `fromRegion` varchar(50) COLLATE utf8mb4_general_ci NOT NULL DEFAULT '', + `taskName` varchar(50) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci NOT NULL, + `taskDesc` varchar(50) COLLATE utf8mb4_general_ci NOT NULL, + `taskState` varchar(50) COLLATE utf8mb4_general_ci NOT NULL DEFAULT '' COMMENT 'taskstate', + `sourceRegion` varchar(50) COLLATE utf8mb4_general_ci DEFAULT NULL, + `targetRegion` varchar(50) COLLATE utf8mb4_general_ci DEFAULT NULL, `createUid` varchar(50) COLLATE utf8mb4_general_ci NOT NULL DEFAULT '', `updateUid` varchar(50) COLLATE utf8mb4_general_ci NOT NULL DEFAULT '', `createTime` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, diff --git a/eventmesh-admin-server/conf/mapper/EventMeshJobInfoMapper.xml b/eventmesh-admin-server/conf/mapper/EventMeshJobInfoMapper.xml index 02e8806680..a053d1c838 100644 --- a/eventmesh-admin-server/conf/mapper/EventMeshJobInfoMapper.xml +++ b/eventmesh-admin-server/conf/mapper/EventMeshJobInfoMapper.xml @@ -19,31 +19,33 @@ --> + PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" + "http://mybatis.org/dtd/mybatis-3-mapper.dtd"> - - - - - - - - - - - - - - + + + + + + + + + + + + + + + - id,jobID,desc, + id,jobID,jobDesc, taskID,transportType,sourceData, - targetData,state,jobType, - fromRegion,createTime,updateTime + targetData,jobState,jobType, + fromRegion,runningRegion,createUid, + updateUid,createTime,updateTime diff --git a/eventmesh-admin-server/conf/mapper/EventMeshTaskInfoMapper.xml b/eventmesh-admin-server/conf/mapper/EventMeshTaskInfoMapper.xml index 05b1dc52a0..c3514fd945 100644 --- a/eventmesh-admin-server/conf/mapper/EventMeshTaskInfoMapper.xml +++ b/eventmesh-admin-server/conf/mapper/EventMeshTaskInfoMapper.xml @@ -26,10 +26,11 @@ - - - - + + + + + @@ -37,8 +38,8 @@ - id,taskID,name, - desc,state,fromRegion, + id,taskID,taskName, + taskDesc,taskState,sourceRegion,targetRegion, createUid,updateUid,createTime, updateTime diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/AdminServerProperties.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/AdminServerProperties.java index 2162731e21..612d398078 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/AdminServerProperties.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/AdminServerProperties.java @@ -17,6 +17,9 @@ package org.apache.eventmesh.admin.server; +import java.util.List; +import java.util.Map; + import org.springframework.boot.context.properties.ConfigurationProperties; import lombok.Getter; @@ -32,4 +35,6 @@ public class AdminServerProperties { private String configurationPath; private String configurationFile; private String serviceName; + private Map> adminServerList; + private String region; } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/HttpServer.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/HttpServer.java index bd896d546c..a5daac881e 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/HttpServer.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/HttpServer.java @@ -24,18 +24,21 @@ import org.springframework.http.ResponseEntity; 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 com.alibaba.druid.support.json.JSONUtils; + @RestController @RequestMapping("/eventmesh/admin") public class HttpServer { @Autowired private TaskBizService taskService; - @RequestMapping("/createTask") - public ResponseEntity> createOrUpdateTask(@RequestBody CreateTaskRequest task) { + @RequestMapping(value = "/createTask", method = RequestMethod.POST) + public ResponseEntity createOrUpdateTask(@RequestBody CreateTaskRequest task) { String uuid = taskService.createTask(task); - return ResponseEntity.ok(Response.success(uuid)); + return ResponseEntity.ok(JSONUtils.toJSONString(Response.success(uuid))); } public boolean deleteTask(Long id) { diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshJobInfo.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshJobInfo.java index 23db5f6c2b..a77eaaaca2 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshJobInfo.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshJobInfo.java @@ -37,7 +37,7 @@ public class EventMeshJobInfo implements Serializable { private String jobID; - private String desc; + private String jobDesc; private String taskID; @@ -47,12 +47,16 @@ public class EventMeshJobInfo implements Serializable { private Integer targetData; - private String state; + private String jobState; private String jobType; + // job request from region private String fromRegion; + // job actually running region + private String runningRegion; + private String createUid; private String updateUid; diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshTaskInfo.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshTaskInfo.java index 5d1b6648c9..2d40f4a082 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshTaskInfo.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshTaskInfo.java @@ -37,13 +37,15 @@ public class EventMeshTaskInfo implements Serializable { private String taskID; - private String name; + private String taskName; - private String desc; + private String taskDesc; - private String state; + private String taskState; - private String fromRegion; + private String sourceRegion; + + private String targetRegion; private String createUid; diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/mapper/EventMeshJobInfoExtMapper.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/mapper/EventMeshJobInfoExtMapper.java index 7f46dcab41..c04c4e3748 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/mapper/EventMeshJobInfoExtMapper.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/mapper/EventMeshJobInfoExtMapper.java @@ -21,11 +21,12 @@ import org.apache.ibatis.annotations.Insert; import org.apache.ibatis.annotations.Mapper; -import org.apache.ibatis.annotations.Options; import org.apache.ibatis.annotations.Param; import java.util.List; +import org.springframework.transaction.annotation.Transactional; + import com.baomidou.mybatisplus.core.mapper.BaseMapper; /** @@ -33,9 +34,18 @@ */ @Mapper public interface EventMeshJobInfoExtMapper extends BaseMapper { - @Insert("insert into event_mesh_job_info(`taskID`,`state`,`jobType`) values" - + "(#{job.taskID},#{job.state},#{job.jobType})") - @Options(useGeneratedKeys = true, keyProperty = "jobID") + + @Insert("") + @Transactional(rollbackFor = Exception.class) int saveBatch(@Param("jobs") List jobInfoList); } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshVerifyServiceImpl.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshVerifyServiceImpl.java new file mode 100644 index 0000000000..5e49ba32ea --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshVerifyServiceImpl.java @@ -0,0 +1,39 @@ +/* + * 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.eventmesh.admin.server.web.db.service.impl; + +import org.apache.eventmesh.admin.server.web.db.entity.EventMeshVerify; +import org.apache.eventmesh.admin.server.web.db.mapper.EventMeshVerifyMapper; +import org.apache.eventmesh.admin.server.web.db.service.EventMeshVerifyService; + +import org.springframework.stereotype.Service; + +import com.baomidou.mybatisplus.extension.service.impl.ServiceImpl; + +/** + * event_mesh_verify + */ +@Service +public class EventMeshVerifyServiceImpl extends ServiceImpl + implements EventMeshVerifyService { + +} + + + + diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/JobDetail.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/JobDetail.java index c47b284483..0e2fa64878 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/JobDetail.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/JobDetail.java @@ -34,7 +34,7 @@ public class JobDetail { private String jobID; - private String desc; + private String jobDesc; private String taskID; @@ -50,7 +50,11 @@ public class JobDetail { private String updateUid; - private String region; + // job request from region + private String fromRegion; + + // job actually running region + private String runningRegion; private DataSource sourceDataSource; diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/job/JobInfoBizService.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/job/JobInfoBizService.java index 9affa10e60..ea02658481 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/job/JobInfoBizService.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/job/JobInfoBizService.java @@ -17,6 +17,7 @@ package org.apache.eventmesh.admin.server.web.service.job; +import org.apache.eventmesh.admin.server.AdminServerProperties; import org.apache.eventmesh.admin.server.AdminServerRuntimeException; import org.apache.eventmesh.admin.server.web.db.entity.EventMeshDataSource; import org.apache.eventmesh.admin.server.web.db.entity.EventMeshJobInfo; @@ -70,12 +71,15 @@ public class JobInfoBizService { @Autowired private PositionBizService positionBizService; + @Autowired + private AdminServerProperties properties; + public boolean updateJobState(String jobID, TaskState state) { if (jobID == null || state == null) { return false; } EventMeshJobInfo jobInfo = new EventMeshJobInfo(); - jobInfo.setState(state.name()); + jobInfo.setJobState(state.name()); return jobInfoService.update(jobInfo, Wrappers.update().eq("jobID", jobID).ne("state", TaskState.DELETE.name())); } @@ -86,34 +90,40 @@ public List createJobs(List jobs) { return null; } List entityList = new LinkedList<>(); + for (JobDetail job : jobs) { + // if running region not equal with admin region continue + if (!job.getRunningRegion().equals(properties.getRegion())) { + continue; + } EventMeshJobInfo entity = new EventMeshJobInfo(); - entity.setState(TaskState.INIT.name()); + entity.setJobState(TaskState.INIT.name()); entity.setTaskID(job.getTaskID()); entity.setJobType(job.getJobType().name()); - entity.setDesc(job.getDesc()); + entity.setJobDesc(job.getJobDesc()); String jobID = UUID.randomUUID().toString(); entity.setJobID(jobID); entity.setTransportType(job.getTransportType().name()); entity.setCreateUid(job.getCreateUid()); entity.setUpdateUid(job.getUpdateUid()); - entity.setFromRegion(job.getRegion()); + entity.setFromRegion(job.getFromRegion()); + entity.setRunningRegion(job.getRunningRegion()); CreateOrUpdateDataSourceReq source = new CreateOrUpdateDataSourceReq(); source.setType(job.getTransportType().getSrc()); source.setOperator(job.getCreateUid()); - source.setRegion(job.getRegion()); + source.setRegion(job.getSourceDataSource().getRegion()); source.setDesc(job.getSourceConnectorDesc()); - source.setConfig(job.getSourceDataSource()); + source.setConfig(job.getSourceDataSource().getConf()); EventMeshDataSource createdSource = dataSourceBizService.createDataSource(source); entity.setSourceData(createdSource.getId()); CreateOrUpdateDataSourceReq sink = new CreateOrUpdateDataSourceReq(); sink.setType(job.getTransportType().getDst()); sink.setOperator(job.getCreateUid()); - sink.setRegion(job.getRegion()); + sink.setRegion(job.getSinkDataSource().getRegion()); sink.setDesc(job.getSinkConnectorDesc()); - sink.setConfig(job.getSinkDataSource()); - EventMeshDataSource createdSink = dataSourceBizService.createDataSource(source); + sink.setConfig(job.getSinkDataSource().getConf()); + EventMeshDataSource createdSink = dataSourceBizService.createDataSource(sink); entity.setTargetData(createdSink.getId()); entityList.add(entity); @@ -167,7 +177,7 @@ public JobDetail getJobDetail(String jobID) { detail.setSinkConnectorDesc(target.getDescription()); } - TaskState state = TaskState.fromIndex(job.getState()); + TaskState state = TaskState.fromIndex(job.getJobState()); if (state == null) { throw new AdminServerRuntimeException(ErrorCode.BAD_DB_DATA, "illegal job state in db"); } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/task/TaskBizService.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/task/TaskBizService.java index b4fdc57af0..f686456135 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/task/TaskBizService.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/task/TaskBizService.java @@ -17,6 +17,7 @@ package org.apache.eventmesh.admin.server.web.service.task; +import org.apache.eventmesh.admin.server.AdminServerProperties; import org.apache.eventmesh.admin.server.web.db.entity.EventMeshTaskInfo; import org.apache.eventmesh.admin.server.web.db.service.EventMeshTaskInfoService; import org.apache.eventmesh.admin.server.web.pojo.JobDetail; @@ -24,13 +25,18 @@ import org.apache.eventmesh.common.remote.TaskState; import org.apache.eventmesh.common.remote.request.CreateTaskRequest; +import org.apache.commons.lang3.StringUtils; + import java.util.List; +import java.util.Random; import java.util.UUID; import java.util.stream.Collectors; import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.http.ResponseEntity; import org.springframework.stereotype.Service; import org.springframework.transaction.annotation.Transactional; +import org.springframework.web.client.RestTemplate; @Service public class TaskBizService { @@ -40,38 +46,67 @@ public class TaskBizService { @Autowired private JobInfoBizService jobInfoService; + @Autowired + private AdminServerProperties properties; + @Transactional public String createTask(CreateTaskRequest req) { - String taskID = UUID.randomUUID().toString(); + String taskID = req.getTaskId(); + if (StringUtils.isEmpty(taskID)) { + taskID = UUID.randomUUID().toString(); + req.setTaskId(taskID); + } + + String targetRegion = req.getTargetRegion(); + // not from other admin && target not equals with self region + if (!req.isFlag() && !StringUtils.equals(properties.getRegion(), targetRegion)) { + List adminServerList = properties.getAdminServerList().get(targetRegion); + if (adminServerList == null || adminServerList.isEmpty()) { + throw new RuntimeException("No admin server available for region: " + targetRegion); + } + String targetUrl = adminServerList.get(new Random().nextInt(adminServerList.size())) + "/eventmesh/admin/createTask"; + + RestTemplate restTemplate = new RestTemplate(); + req.setFlag(true); + ResponseEntity response = restTemplate.postForEntity(targetUrl, req, String.class); + if (!response.getStatusCode().is2xxSuccessful()) { + throw new RuntimeException("Failed to create task on admin server: " + targetUrl); + } + } + + String finalTaskID = taskID; List jobs = req.getJobs().stream().map(x -> { JobDetail job = parse(x); - job.setTaskID(taskID); - job.setRegion(req.getRegion()); + job.setTaskID(finalTaskID); job.setCreateUid(req.getUid()); job.setUpdateUid(req.getUid()); return job; }).collect(Collectors.toList()); jobInfoService.createJobs(jobs); EventMeshTaskInfo taskInfo = new EventMeshTaskInfo(); - taskInfo.setTaskID(taskID); - taskInfo.setName(req.getName()); - taskInfo.setDesc(req.getDesc()); - taskInfo.setState(TaskState.INIT.name()); + taskInfo.setTaskID(finalTaskID); + taskInfo.setTaskName(req.getTaskName()); + taskInfo.setTaskDesc(req.getTaskDesc()); + taskInfo.setTaskState(TaskState.INIT.name()); taskInfo.setCreateUid(req.getUid()); - taskInfo.setFromRegion(req.getRegion()); + taskInfo.setSourceRegion(req.getSourceRegion()); + taskInfo.setTargetRegion(req.getTargetRegion()); taskInfoService.save(taskInfo); - return taskID; + return finalTaskID; } private JobDetail parse(CreateTaskRequest.JobDetail src) { JobDetail dst = new JobDetail(); - dst.setDesc(src.getDesc()); + dst.setJobDesc(src.getJobDesc()); dst.setTransportType(src.getTransportType()); dst.setSourceConnectorDesc(src.getSourceConnectorDesc()); dst.setSourceDataSource(src.getSourceDataSource()); dst.setSinkConnectorDesc(src.getSinkConnectorDesc()); dst.setSinkDataSource(src.getSinkDataSource()); + // full/increase/check dst.setJobType(src.getJobType()); + dst.setFromRegion(src.getFromRegion()); + dst.setRunningRegion(src.getRunningRegion()); return dst; } } diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/TransportType.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/TransportType.java index 95a88a23fa..82e7bc021d 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/TransportType.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/TransportType.java @@ -30,8 +30,12 @@ public enum TransportType { REDIS_REDIS(DataSourceType.REDIS, DataSourceType.REDIS), ROCKETMQ_ROCKETMQ(DataSourceType.ROCKETMQ, DataSourceType.ROCKETMQ), MYSQL_HTTP(DataSourceType.MYSQL, DataSourceType.HTTP), + ROCKETMQ_HTTP(DataSourceType.ROCKETMQ, DataSourceType.HTTP), HTTP_MYSQL(DataSourceType.HTTP, DataSourceType.MYSQL), - REDIS_MQ(DataSourceType.REDIS, DataSourceType.ROCKETMQ); + HTTP_REDIS(DataSourceType.HTTP, DataSourceType.REDIS), + HTTP_ROCKETMQ(DataSourceType.HTTP, DataSourceType.ROCKETMQ), + REDIS_MQ(DataSourceType.REDIS, DataSourceType.ROCKETMQ), + ; private static final Map INDEX_TYPES = new HashMap<>(); private static final TransportType[] TYPES = TransportType.values(); private static final String SEPARATOR = "@"; diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSource.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSource.java index 7af3812f24..afda984805 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSource.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSource.java @@ -17,27 +17,30 @@ package org.apache.eventmesh.common.remote.datasource; +import org.apache.eventmesh.common.config.connector.Config; +import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSinkConfig; +import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSourceConfig; + import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import lombok.Getter; +import lombok.Data; -@Getter +@Data public class DataSource { - private final DataSourceType type; + + private DataSourceType type; + private String desc; @JsonTypeInfo(use = JsonTypeInfo.Id.CLASS) @JsonSubTypes({ - @JsonSubTypes.Type(value = MySqlIncDataSourceSourceConf.class, name = "MySqlIncDataSourceSourceConf") + @JsonSubTypes.Type(value = CanalSourceConfig.class, name = "CanalSourceConfig"), + @JsonSubTypes.Type(value = CanalSinkConfig.class, name = "CanalSinkConfig") }) - private final DataSourceConf conf; - private final Class confClazz; + private Config conf; - public DataSource(DataSourceType type, DataSourceConf conf) { - this.type = type; - this.conf = conf; - this.confClazz = conf.getConfClass(); - } + private Class confClazz; + private String region; } diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/MySqlIncDataSourceSourceConf.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/MySqlIncDataSourceSourceConf.java deleted file mode 100644 index f8c825e963..0000000000 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/MySqlIncDataSourceSourceConf.java +++ /dev/null @@ -1,85 +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.eventmesh.common.remote.datasource; - -import org.apache.eventmesh.common.config.connector.rdb.canal.SourceConnectorConfig; -import org.apache.eventmesh.common.remote.job.SyncConsistency; -import org.apache.eventmesh.common.remote.job.SyncMode; -import org.apache.eventmesh.common.remote.offset.RecordPosition; - -import java.util.List; - -public class MySqlIncDataSourceSourceConf extends DataSourceConf { - @Override - public Class getConfClass() { - return MySqlIncDataSourceSourceConf.class; - } - - private String destination; - - private Long canalInstanceId; - - private String desc; - - private boolean ddlSync = true; - - private boolean filterTableError = false; - - private Long slaveId; - - private Short clientId; - - private String serverUUID; - - private boolean isMariaDB = true; - - private boolean isGTIDMode = true; - - private Integer batchSize = 10000; - - private Long batchTimeout = -1L; - - private String tableFilter; - - private String fieldFilter; - - private List recordPositions; - - // ================================= channel parameter - // ================================ - - // enable remedy - private Boolean enableRemedy = false; - - // sync mode: field/row - private SyncMode syncMode; - - // sync consistency - private SyncConsistency syncConsistency; - - // ================================= system parameter - // ================================ - - // Column name of the bidirectional synchronization mark - private String needSyncMarkTableColumnName = "needSync"; - - // Column value of the bidirectional synchronization mark - private String needSyncMarkTableColumnValue = "needSync"; - - private SourceConnectorConfig sourceConnectorConfig; -} diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/CreateOrUpdateDataSourceReq.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/CreateOrUpdateDataSourceReq.java index 4ecf9b4527..fadfa68e75 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/CreateOrUpdateDataSourceReq.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/CreateOrUpdateDataSourceReq.java @@ -17,7 +17,7 @@ package org.apache.eventmesh.common.remote.request; -import org.apache.eventmesh.common.remote.datasource.DataSource; +import org.apache.eventmesh.common.config.connector.Config; import org.apache.eventmesh.common.remote.datasource.DataSourceType; import lombok.Data; @@ -29,10 +29,11 @@ @Data @EqualsAndHashCode(callSuper = true) public class CreateOrUpdateDataSourceReq extends BaseRemoteRequest { + private Integer id; private DataSourceType type; private String desc; - private DataSource config; + private Config config; private String region; private String operator; } diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/CreateTaskRequest.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/CreateTaskRequest.java index ce24e03416..47c45595af 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/CreateTaskRequest.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/CreateTaskRequest.java @@ -30,16 +30,35 @@ */ @Data public class CreateTaskRequest { - private String name; - private String desc; + + private String taskId; + + // task name + private String taskName; + + // task description + private String taskDesc; + + // task owner or updater private String uid; + private List jobs; - private String region; + + // task source region + private String sourceRegion; + + // task target region + private String targetRegion; + + // mark request send by other region admin, default is false + private boolean flag = false; @Data public static class JobDetail { - private String desc; + private String jobDesc; + + // full/increase/check private JobType jobType; private DataSource sourceDataSource; @@ -51,5 +70,11 @@ public static class JobDetail { private String sinkConnectorDesc; private TransportType transportType; + + // job request from region + private String fromRegion; + + // job actually running region + private String runningRegion; } } diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/RuntimeInstanceConfig.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/RuntimeInstanceConfig.java index 7171b3fc27..caa5330fe3 100644 --- a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/RuntimeInstanceConfig.java +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/RuntimeInstanceConfig.java @@ -28,6 +28,8 @@ @Config(path = "classPath://runtime.yaml") public class RuntimeInstanceConfig { + private boolean registryEnabled; + private String registryServerAddr; private String registryPluginType; @@ -36,7 +38,7 @@ public class RuntimeInstanceConfig { private String adminServiceName; - private String adminServerAddr; + private String adminServiceAddr; private ComponentType componentType; diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/boot/RuntimeInstance.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/boot/RuntimeInstance.java index 0fade897f6..acea321e95 100644 --- a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/boot/RuntimeInstance.java +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/boot/RuntimeInstance.java @@ -41,11 +41,11 @@ @Slf4j public class RuntimeInstance { - private String adminServerAddr = "127.0.0.1:8081"; + private String adminServiceAddr; private Map adminServerInfoMap = new HashMap<>(); - private final RegistryService registryService; + private RegistryService registryService; private Runtime runtime; @@ -57,29 +57,34 @@ public class RuntimeInstance { public RuntimeInstance(RuntimeInstanceConfig runtimeInstanceConfig) { this.runtimeInstanceConfig = runtimeInstanceConfig; - this.registryService = RegistryFactory.getInstance(runtimeInstanceConfig.getRegistryPluginType()); + if (runtimeInstanceConfig.isRegistryEnabled()) { + this.registryService = RegistryFactory.getInstance(runtimeInstanceConfig.getRegistryPluginType()); + } } public void init() throws Exception { - registryService.init(); - QueryInstances queryInstances = new QueryInstances(); - queryInstances.setServiceName(runtimeInstanceConfig.getAdminServiceName()); - queryInstances.setHealth(true); - List adminServerRegisterInfoList = registryService.selectInstances(queryInstances); - if (!adminServerRegisterInfoList.isEmpty()) { - adminServerAddr = getRandomAdminServerAddr(adminServerRegisterInfoList); - } else { - throw new RuntimeException("admin server address is empty, please check"); + if (registryService != null) { + registryService.init(); + QueryInstances queryInstances = new QueryInstances(); + queryInstances.setServiceName(runtimeInstanceConfig.getAdminServiceName()); + queryInstances.setHealth(true); + List adminServerRegisterInfoList = registryService.selectInstances(queryInstances); + if (!adminServerRegisterInfoList.isEmpty()) { + adminServiceAddr = getRandomAdminServerAddr(adminServerRegisterInfoList); + } else { + throw new RuntimeException("admin server address is empty, please check"); + } + // use registry adminServiceAddr value replace config + runtimeInstanceConfig.setAdminServiceAddr(adminServiceAddr); } - runtimeInstanceConfig.setAdminServerAddr(adminServerAddr); + runtimeFactory = initRuntimeFactory(runtimeInstanceConfig); runtime = runtimeFactory.createRuntime(runtimeInstanceConfig); runtime.init(); } public void start() throws Exception { - if (!StringUtils.isBlank(adminServerAddr)) { - + if (!StringUtils.isBlank(adminServiceAddr) && registryService != null) { registryService.subscribe((event) -> { log.info("runtime receive registry event: {}", event); List registerServerInfoList = event.getInstances(); @@ -91,7 +96,6 @@ public void start() throws Exception { adminServerInfoMap = registerServerInfoMap; updateAdminServerAddr(); } - }, runtimeInstanceConfig.getAdminServiceName()); runtime.start(); isStarted = true; @@ -106,14 +110,14 @@ public void shutdown() throws Exception { private void updateAdminServerAddr() throws Exception { if (isStarted) { - if (!adminServerInfoMap.containsKey(adminServerAddr)) { - adminServerAddr = getRandomAdminServerAddr(adminServerInfoMap); - log.info("admin server address changed to: {}", adminServerAddr); + if (!adminServerInfoMap.containsKey(adminServiceAddr)) { + adminServiceAddr = getRandomAdminServerAddr(adminServerInfoMap); + log.info("admin server address changed to: {}", adminServiceAddr); shutdown(); start(); } } else { - adminServerAddr = getRandomAdminServerAddr(adminServerInfoMap); + adminServiceAddr = getRandomAdminServerAddr(adminServerInfoMap); } } diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java index 6cd0452b83..1e589ebd97 100644 --- a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/connector/ConnectorRuntime.java @@ -150,7 +150,7 @@ public void init() throws Exception { private void initAdminService() { // create gRPC channel - channel = ManagedChannelBuilder.forTarget(runtimeInstanceConfig.getAdminServerAddr()).usePlaintext().build(); + channel = ManagedChannelBuilder.forTarget(runtimeInstanceConfig.getAdminServiceAddr()).usePlaintext().build(); adminServiceStub = AdminServiceGrpc.newStub(channel).withWaitForReady(); diff --git a/eventmesh-runtime-v2/src/main/resources/runtime.yaml b/eventmesh-runtime-v2/src/main/resources/runtime.yaml index 44c5f6f91f..c5ffac9d92 100644 --- a/eventmesh-runtime-v2/src/main/resources/runtime.yaml +++ b/eventmesh-runtime-v2/src/main/resources/runtime.yaml @@ -16,7 +16,9 @@ # componentType: CONNECTOR +registryEnabled: false registryServerAddr: 127.0.0.1:8085 registryPluginType: nacos storagePluginType: memory adminServiceName: eventmesh-admin +adminServiceAddr: "127.0.0.1:8085;127.0.0.1:8086"