From 036c932ca71aa54fe2c383c9735cad4f9bb6ba85 Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Sun, 8 Dec 2024 18:36:59 +0800 Subject: [PATCH 1/5] [ISSUE #5137] update connector runtime v2 module --- .../remote/request/ReportMonitorRequest.java | 38 +++ .../api/monitor/AbstractConnectorMonitor.java | 80 ++++++ .../openconnect/api/monitor/Monitor.java | 30 +++ .../api/monitor/MonitorRegistry.java | 34 +++ .../runtime/boot/RuntimeInstanceStarter.java | 1 - .../runtime/connector/ConnectorRuntime.java | 230 +++++++----------- .../runtime/service/health/HealthService.java | 112 +++++++++ .../service/monitor/MonitorService.java | 144 +++++++++++ .../runtime/service/monitor/SinkMonitor.java | 52 ++++ .../service/monitor/SourceMonitor.java | 47 ++++ .../runtime/service/status/StatusService.java | 94 +++++++ .../runtime/service/verify/VerifyService.java | 138 +++++++++++ .../eventmesh/runtime/util/RuntimeUtils.java | 13 + 13 files changed, 872 insertions(+), 141 deletions(-) create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportMonitorRequest.java create mode 100644 eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/AbstractConnectorMonitor.java create mode 100644 eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/Monitor.java create mode 100644 eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/MonitorRegistry.java create mode 100644 eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/health/HealthService.java create mode 100644 eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/MonitorService.java create mode 100644 eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/SinkMonitor.java create mode 100644 eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/SourceMonitor.java create mode 100644 eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/status/StatusService.java create mode 100644 eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/verify/VerifyService.java diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportMonitorRequest.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportMonitorRequest.java new file mode 100644 index 0000000000..12278df27f --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/ReportMonitorRequest.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.common.remote.request; + +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.ToString; + +@Data +@EqualsAndHashCode(callSuper = true) +@ToString +public class ReportMonitorRequest extends BaseRemoteRequest { + private String taskID; + private String jobID; + private String address; + private String connectorStage; + private String transportType; + private long totalReqNum; + private long totalTimeCost; + private long maxTimeCost; + private long avgTimeCost; + private double tps; +} diff --git a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/AbstractConnectorMonitor.java b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/AbstractConnectorMonitor.java new file mode 100644 index 0000000000..b9205804a4 --- /dev/null +++ b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/AbstractConnectorMonitor.java @@ -0,0 +1,80 @@ +/* + * 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.openconnect.api.monitor; + +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.LongAdder; + +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@Getter +public abstract class AbstractConnectorMonitor implements Monitor { + + private final String taskId; + private final String jobId; + private final String ip; + private final LongAdder totalRecordNum; + private final LongAdder totalTimeCost; + protected final AtomicLong startTime; + private final AtomicLong maxTimeCost; + private long averageTime = 0; + private double tps = 0; + + public AbstractConnectorMonitor(String taskId, String jobId, String ip) { + this.taskId = taskId; + this.jobId = jobId; + this.ip = ip; + this.totalRecordNum = new LongAdder(); + this.totalTimeCost = new LongAdder(); + this.startTime = new AtomicLong(System.currentTimeMillis()); + this.maxTimeCost = new AtomicLong(); + } + + @Override + public synchronized void recordProcess(long timeCost) { + totalRecordNum.increment(); + totalTimeCost.add(timeCost); + maxTimeCost.updateAndGet(max -> Math.max(max, timeCost)); + } + + @Override + public synchronized void recordProcess(int recordCount, long timeCost) { + totalRecordNum.add(recordCount); + totalTimeCost.add(timeCost); + maxTimeCost.updateAndGet(max -> Math.max(max, timeCost)); + } + + @Override + public synchronized void printMetrics() { + long totalRecords = totalRecordNum.sum(); + long totalCost = totalTimeCost.sum(); + averageTime = totalRecords > 0 ? totalCost / totalRecords : 0; + long elapsedTime = (System.currentTimeMillis() - startTime.get()) / 1000; // in seconds + tps = elapsedTime > 0 ? (double) totalRecords / elapsedTime : 0; + + log.info("========== Metrics =========="); + log.info("TaskId: {}|JobId: {}|ip: {}", taskId, jobId, ip); + log.info("Total records: {}", totalRecordNum); + log.info("Total time (ms): {}", totalTimeCost); + log.info("Max time per record (ms): {}", maxTimeCost); + log.info("Average time per record (ms): {}", averageTime); + log.info("TPS: {}", tps); + } +} diff --git a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/Monitor.java b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/Monitor.java new file mode 100644 index 0000000000..4d4d9efb0c --- /dev/null +++ b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/Monitor.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.openconnect.api.monitor; + +/** + * Monitor Interface. + * All monitors should implement this interface. + */ +public interface Monitor { + void recordProcess(long timeCost); + + void recordProcess(int recordCount, long timeCost); + + void printMetrics(); +} \ No newline at end of file diff --git a/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/MonitorRegistry.java b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/MonitorRegistry.java new file mode 100644 index 0000000000..904efc5d3f --- /dev/null +++ b/eventmesh-openconnect/eventmesh-openconnect-java/src/main/java/org/apache/eventmesh/openconnect/api/monitor/MonitorRegistry.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.openconnect.api.monitor; + +import java.util.ArrayList; +import java.util.List; + +import lombok.Getter; + +public class MonitorRegistry { + + @Getter + private static final List monitors = new ArrayList<>(); + + public static void registerMonitor(Monitor monitor) { + monitors.add(monitor); + } + +} diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/boot/RuntimeInstanceStarter.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/boot/RuntimeInstanceStarter.java index 42745c8dd7..0881521879 100644 --- a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/boot/RuntimeInstanceStarter.java +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/boot/RuntimeInstanceStarter.java @@ -40,7 +40,6 @@ public static void main(String[] args) { long start = System.currentTimeMillis(); runtimeInstance.shutdown(); long end = System.currentTimeMillis(); - log.info("runtime shutdown cost {}ms", end - start); } catch (Exception e) { log.error("exception when shutdown {}", e.getMessage(), e); 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 3d3c864b58..92e78256ec 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 @@ -33,9 +33,6 @@ import org.apache.eventmesh.common.protocol.grpc.adminserver.Payload; import org.apache.eventmesh.common.remote.JobState; import org.apache.eventmesh.common.remote.request.FetchJobRequest; -import org.apache.eventmesh.common.remote.request.ReportHeartBeatRequest; -import org.apache.eventmesh.common.remote.request.ReportJobRequest; -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; @@ -57,33 +54,34 @@ import org.apache.eventmesh.openconnect.util.ConfigUtil; import org.apache.eventmesh.runtime.Runtime; import org.apache.eventmesh.runtime.RuntimeInstanceConfig; +import org.apache.eventmesh.runtime.service.health.HealthService; +import org.apache.eventmesh.runtime.service.monitor.MonitorService; +import org.apache.eventmesh.runtime.service.monitor.SinkMonitor; +import org.apache.eventmesh.runtime.service.monitor.SourceMonitor; +import org.apache.eventmesh.runtime.service.status.StatusService; +import org.apache.eventmesh.runtime.service.verify.VerifyService; +import org.apache.eventmesh.runtime.util.RuntimeUtils; 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; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.Random; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import io.grpc.ManagedChannel; import io.grpc.ManagedChannelBuilder; -import io.grpc.stub.StreamObserver; import com.google.protobuf.Any; import com.google.protobuf.UnsafeByteOperations; @@ -103,10 +101,6 @@ public class ConnectorRuntime implements Runtime { private AdminServiceBlockingStub adminServiceBlockingStub; - StreamObserver responseObserver; - - StreamObserver requestObserver; - private Source sourceConnector; private Sink sinkConnector; @@ -129,9 +123,6 @@ public class ConnectorRuntime implements Runtime { private final ExecutorService sinkService = ThreadPoolFactory.createSingleExecutor("eventMesh-sinkService"); - private final ScheduledExecutorService heartBeatExecutor = Executors.newSingleThreadScheduledExecutor(); - - private final ExecutorService reportVerifyExecutor = Executors.newSingleThreadExecutor(); private final BlockingQueue queue; @@ -143,6 +134,18 @@ public class ConnectorRuntime implements Runtime { private String adminServerAddr; + private HealthService healthService; + + private MonitorService monitorService; + + private SourceMonitor sourceMonitor; + + private SinkMonitor sinkMonitor; + + private VerifyService verifyService; + + private StatusService statusService; + public ConnectorRuntime(RuntimeInstanceConfig runtimeInstanceConfig) { this.runtimeInstanceConfig = runtimeInstanceConfig; @@ -156,46 +159,31 @@ public void init() throws Exception { initStorageService(); + initStatusService(); + initConnectorService(); + + initMonitorService(); + + initHealthService(); + + initVerfiyService(); + } private void initAdminService() { - adminServerAddr = getRandomAdminServerAddr(runtimeInstanceConfig.getAdminServiceAddr()); + adminServerAddr = RuntimeUtils.getRandomAdminServerAddr(runtimeInstanceConfig.getAdminServiceAddr()); // create gRPC channel - channel = ManagedChannelBuilder.forTarget(adminServerAddr).usePlaintext().build(); + channel = ManagedChannelBuilder.forTarget(adminServerAddr) + .usePlaintext() + .enableRetry() + .maxRetryAttempts(3) + .build(); adminServiceStub = AdminServiceGrpc.newStub(channel).withWaitForReady(); adminServiceBlockingStub = AdminServiceGrpc.newBlockingStub(channel).withWaitForReady(); - responseObserver = new StreamObserver() { - @Override - public void onNext(Payload response) { - log.info("runtime receive message: {} ", response); - } - - @Override - public void onError(Throwable t) { - log.error("runtime receive error message: {}", t.getMessage()); - } - - @Override - public void onCompleted() { - log.info("runtime finished receive message and completed"); - } - }; - - requestObserver = adminServiceStub.invokeBiStream(responseObserver); - } - - private String getRandomAdminServerAddr(String adminServerAddrList) { - String[] addresses = adminServerAddrList.split(";"); - if (addresses.length == 0) { - throw new IllegalArgumentException("Admin server address list is empty"); - } - Random random = new Random(); - int randomIndex = random.nextInt(addresses.length); - return addresses[randomIndex]; } private void initStorageService() { @@ -206,11 +194,16 @@ private void initStorageService() { } + private void initStatusService() { + statusService = new StatusService(adminServiceStub, adminServiceBlockingStub); + } + private void initConnectorService() throws Exception { connectorRuntimeConfig = ConfigService.getInstance().buildConfigInstance(ConnectorRuntimeConfig.class); FetchJobResponse jobResponse = fetchJobConfig(); + log.info("fetch job config from admin server: {}", JsonUtils.toJSONString(jobResponse)); if (jobResponse == null) { isFailed = true; @@ -271,7 +264,7 @@ private void initConnectorService() throws Exception { sinkConnectorContext.setJobType(jobResponse.getType()); sinkConnector.init(sinkConnectorContext); - reportJobRequest(connectorRuntimeConfig.getJobID(), JobState.INIT); + statusService.reportJobStatus(connectorRuntimeConfig.getJobID(), JobState.INIT); } @@ -292,27 +285,31 @@ private FetchJobResponse fetchJobConfig() { return null; } - @Override - public void start() throws Exception { + private void initMonitorService() { + monitorService = new MonitorService(adminServiceStub, adminServiceBlockingStub); + sourceMonitor = new SourceMonitor(connectorRuntimeConfig.getTaskID(), connectorRuntimeConfig.getJobID(), IPUtils.getLocalAddress()); + monitorService.registerMonitor(sourceMonitor); + sinkMonitor = new SinkMonitor(connectorRuntimeConfig.getTaskID(), connectorRuntimeConfig.getJobID(), IPUtils.getLocalAddress()); + monitorService.registerMonitor(sinkMonitor); + } - heartBeatExecutor.scheduleAtFixedRate(() -> { + private void initHealthService() { + healthService = new HealthService(adminServiceStub, adminServiceBlockingStub, connectorRuntimeConfig); + } - ReportHeartBeatRequest heartBeat = new ReportHeartBeatRequest(); - heartBeat.setAddress(IPUtils.getLocalAddress()); - heartBeat.setReportedTimeStamp(String.valueOf(System.currentTimeMillis())); - heartBeat.setJobID(connectorRuntimeConfig.getJobID()); + private void initVerfiyService() { + verifyService = new VerifyService(adminServiceStub, adminServiceBlockingStub, connectorRuntimeConfig); + } - Metadata metadata = Metadata.newBuilder().setType(ReportHeartBeatRequest.class.getSimpleName()).build(); + @Override + public void start() throws Exception { + // start offsetMgmtService + offsetManagementService.start(); - Payload request = Payload.newBuilder().setMetadata(metadata) - .setBody(Any.newBuilder().setValue(UnsafeByteOperations.unsafeWrap(Objects.requireNonNull(JsonUtils.toJSONBytes(heartBeat)))).build()) - .build(); + monitorService.start(); - requestObserver.onNext(request); - }, 5, 5, TimeUnit.SECONDS); + healthService.start(); - // start offsetMgmtService - offsetManagementService.start(); isRunning = true; // start sinkService sinkService.execute(() -> { @@ -320,32 +317,34 @@ public void start() throws Exception { startSinkConnector(); } catch (Exception e) { isFailed = true; - log.error("sink connector [{}] start fail", sinkConnector.name(), e); + log.error("sink connector start fail", e.getStackTrace()); try { this.stop(); } catch (Exception ex) { log.error("Failed to stop after exception", ex); } - throw new RuntimeException(e); + } finally { + System.exit(-1); } }); - // start + // start sourceService sourceService.execute(() -> { try { startSourceConnector(); } catch (Exception e) { isFailed = true; - log.error("source connector [{}] start fail", sourceConnector.name(), e); + log.error("source connector start fail", e); try { this.stop(); } catch (Exception ex) { log.error("Failed to stop after exception", ex); } - throw new RuntimeException(e); + } finally { + System.exit(-1); } }); - reportJobRequest(connectorRuntimeConfig.getJobID(), JobState.RUNNING); + statusService.reportJobStatus(connectorRuntimeConfig.getJobID(), JobState.RUNNING); } @Override @@ -353,26 +352,30 @@ public void stop() throws Exception { log.info("ConnectorRuntime start stop"); isRunning = false; if (isFailed) { - reportJobRequest(connectorRuntimeConfig.getJobID(), JobState.FAIL); + statusService.reportJobStatus(connectorRuntimeConfig.getJobID(), JobState.FAIL); } else { - reportJobRequest(connectorRuntimeConfig.getJobID(), JobState.COMPLETE); + statusService.reportJobStatus(connectorRuntimeConfig.getJobID(), JobState.COMPLETE); } sourceConnector.stop(); sinkConnector.stop(); + monitorService.stop(); + healthService.stop(); sourceService.shutdown(); sinkService.shutdown(); - heartBeatExecutor.shutdown(); - reportVerifyExecutor.shutdown(); - requestObserver.onCompleted(); + verifyService.stop(); + statusService.stop(); if (channel != null && !channel.isShutdown()) { - channel.shutdown(); + channel.shutdown().awaitTermination(5, TimeUnit.SECONDS); } + log.info("ConnectorRuntime stopped"); } private void startSourceConnector() throws Exception { sourceConnector.start(); while (isRunning) { + long sourceStartTime = System.currentTimeMillis(); List connectorRecordList = sourceConnector.poll(); + long sinkStartTime = System.currentTimeMillis(); // TODO: use producer pub record to storage replace below if (connectorRecordList != null && !connectorRecordList.isEmpty()) { for (ConnectRecord record : connectorRecordList) { @@ -381,19 +384,14 @@ private void startSourceConnector() throws Exception { record.addExtension("recordUniqueId", record.getRecordId()); } - queue.put(record); - - // if enabled incremental data reporting consistency check - if (connectorRuntimeConfig.enableIncrementalDataConsistencyCheck) { - 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) { log.debug("send record to sink callback success, record: {}", record); + long sinkEndTime = System.currentTimeMillis(); + sinkMonitor.recordProcess(sinkEndTime - sinkStartTime); // commit record sourceConnector.commit(record); if (record.getPosition() != null) { @@ -424,6 +422,16 @@ public void onException(SendExceptionContext sendExceptionContext) { } } }); + + queue.put(record); + long sourceEndTime = System.currentTimeMillis(); + sourceMonitor.recordProcess(sourceEndTime - sourceStartTime); + + // if enabled incremental data reporting consistency check + if (connectorRuntimeConfig.enableIncrementalDataConsistencyCheck) { + verifyService.reportVerifyRequest(record, ConnectorStage.SOURCE); + } + } } } @@ -438,64 +446,6 @@ private SendResult convertToSendResult(ConnectRecord record) { return result; } - private void reportVerifyRequest(ConnectRecord record, ConnectorRuntimeConfig connectorRuntimeConfig, ConnectorStage connectorStage) { - reportVerifyExecutor.submit(() -> { - try { - // use record data + recordUniqueId for md5 - String md5Str = md5(record.getData().toString() + record.getExtension("recordUniqueId")); - ReportVerifyRequest reportVerifyRequest = new ReportVerifyRequest(); - reportVerifyRequest.setTaskID(connectorRuntimeConfig.getTaskID()); - reportVerifyRequest.setJobID(connectorRuntimeConfig.getJobID()); - reportVerifyRequest.setRecordID(record.getRecordId()); - 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); - } catch (Exception e) { - log.error("Failed to report verify request", e); - } - }); - } - - private void reportJobRequest(String jobId, JobState jobState) throws InterruptedException { - ReportJobRequest reportJobRequest = new ReportJobRequest(); - reportJobRequest.setJobID(jobId); - reportJobRequest.setState(jobState); - Metadata metadata = Metadata.newBuilder() - .setType(ReportJobRequest.class.getSimpleName()) - .build(); - Payload payload = Payload.newBuilder() - .setMetadata(metadata) - .setBody(Any.newBuilder().setValue(UnsafeByteOperations.unsafeWrap(Objects.requireNonNull(JsonUtils.toJSONBytes(reportJobRequest)))) - .build()) - .build(); - requestObserver.onNext(payload); - } - - 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())); @@ -589,7 +539,7 @@ private void startSinkConnector() throws Exception { sinkConnector.put(connectRecordList); // if enabled incremental data reporting consistency check if (connectorRuntimeConfig.enableIncrementalDataConsistencyCheck) { - reportVerifyRequest(connectRecord, connectorRuntimeConfig, ConnectorStage.SINK); + verifyService.reportVerifyRequest(connectRecord, ConnectorStage.SINK); } } } diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/health/HealthService.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/health/HealthService.java new file mode 100644 index 0000000000..54f924874b --- /dev/null +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/health/HealthService.java @@ -0,0 +1,112 @@ +/* + * 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.runtime.service.health; + +import org.apache.eventmesh.common.protocol.grpc.adminserver.AdminServiceGrpc; +import org.apache.eventmesh.common.protocol.grpc.adminserver.Metadata; +import org.apache.eventmesh.common.protocol.grpc.adminserver.Payload; +import org.apache.eventmesh.common.remote.request.ReportHeartBeatRequest; +import org.apache.eventmesh.common.utils.IPUtils; +import org.apache.eventmesh.common.utils.JsonUtils; +import org.apache.eventmesh.runtime.connector.ConnectorRuntimeConfig; + +import java.util.Objects; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import io.grpc.stub.StreamObserver; + +import com.google.protobuf.Any; +import com.google.protobuf.UnsafeByteOperations; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class HealthService { + + private final ScheduledExecutorService scheduler; + + private StreamObserver requestObserver; + + private StreamObserver responseObserver; + + private AdminServiceGrpc.AdminServiceStub adminServiceStub; + + private AdminServiceGrpc.AdminServiceBlockingStub adminServiceBlockingStub; + + private ConnectorRuntimeConfig connectorRuntimeConfig; + + + public HealthService(AdminServiceGrpc.AdminServiceStub adminServiceStub, AdminServiceGrpc.AdminServiceBlockingStub adminServiceBlockingStub, + ConnectorRuntimeConfig connectorRuntimeConfig) { + this.adminServiceStub = adminServiceStub; + this.adminServiceBlockingStub = adminServiceBlockingStub; + this.connectorRuntimeConfig = connectorRuntimeConfig; + + this.scheduler = Executors.newSingleThreadScheduledExecutor(); + + responseObserver = new StreamObserver() { + @Override + public void onNext(Payload response) { + log.debug("health service receive message: {}|{} ", response.getMetadata(), response.getBody()); + } + + @Override + public void onError(Throwable t) { + log.error("health service receive error message: {}", t.getMessage()); + } + + @Override + public void onCompleted() { + log.info("health service finished receive message and completed"); + } + }; + requestObserver = this.adminServiceStub.invokeBiStream(responseObserver); + } + + public void start() { + this.healthReport(); + } + + public void healthReport() { + scheduler.scheduleAtFixedRate(() -> { + ReportHeartBeatRequest heartBeat = new ReportHeartBeatRequest(); + heartBeat.setAddress(IPUtils.getLocalAddress()); + heartBeat.setReportedTimeStamp(String.valueOf(System.currentTimeMillis())); + heartBeat.setJobID(connectorRuntimeConfig.getJobID()); + + Metadata metadata = Metadata.newBuilder().setType(ReportHeartBeatRequest.class.getSimpleName()).build(); + + Payload request = Payload.newBuilder().setMetadata(metadata) + .setBody(Any.newBuilder().setValue(UnsafeByteOperations.unsafeWrap(Objects.requireNonNull(JsonUtils.toJSONBytes(heartBeat)))).build()) + .build(); + + requestObserver.onNext(request); + }, 5, 5, TimeUnit.SECONDS); + } + + + public void stop() { + scheduler.shutdown(); + if (requestObserver != null) { + requestObserver.onCompleted(); + } + } + +} diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/MonitorService.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/MonitorService.java new file mode 100644 index 0000000000..f5af7596c3 --- /dev/null +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/MonitorService.java @@ -0,0 +1,144 @@ +/* + * 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.runtime.service.monitor; + +import org.apache.eventmesh.common.protocol.grpc.adminserver.AdminServiceGrpc; +import org.apache.eventmesh.common.protocol.grpc.adminserver.Metadata; +import org.apache.eventmesh.common.protocol.grpc.adminserver.Payload; +import org.apache.eventmesh.common.remote.request.ReportMonitorRequest; +import org.apache.eventmesh.common.utils.JsonUtils; +import org.apache.eventmesh.openconnect.api.monitor.Monitor; +import org.apache.eventmesh.openconnect.api.monitor.MonitorRegistry; + +import java.util.List; +import java.util.Objects; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import io.grpc.stub.StreamObserver; + +import com.google.protobuf.Any; +import com.google.protobuf.UnsafeByteOperations; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class MonitorService { + + private final ScheduledExecutorService scheduler; + + private StreamObserver requestObserver; + + private StreamObserver responseObserver; + + private AdminServiceGrpc.AdminServiceStub adminServiceStub; + + private AdminServiceGrpc.AdminServiceBlockingStub adminServiceBlockingStub; + + + public MonitorService(AdminServiceGrpc.AdminServiceStub adminServiceStub, AdminServiceGrpc.AdminServiceBlockingStub adminServiceBlockingStub) { + this.adminServiceStub = adminServiceStub; + this.adminServiceBlockingStub = adminServiceBlockingStub; + + this.scheduler = Executors.newSingleThreadScheduledExecutor(); + + responseObserver = new StreamObserver() { + @Override + public void onNext(Payload response) { + log.debug("monitor service receive message: {}|{} ", response.getMetadata(), response.getBody()); + } + + @Override + public void onError(Throwable t) { + log.error("monitor service receive error message: {}", t.getMessage()); + } + + @Override + public void onCompleted() { + log.info("monitor service finished receive message and completed"); + } + }; + requestObserver = this.adminServiceStub.invokeBiStream(responseObserver); + } + + public void registerMonitor(Monitor monitor) { + MonitorRegistry.registerMonitor(monitor); + } + + public void start() { + this.startReporting(); + } + + public void startReporting() { + scheduler.scheduleAtFixedRate(() -> { + List monitors = MonitorRegistry.getMonitors(); + for (Monitor monitor : monitors) { + monitor.printMetrics(); + reportToAdminService(monitor); + } + }, 5, 30, TimeUnit.SECONDS); + } + + private void reportToAdminService(Monitor monitor) { + ReportMonitorRequest request = new ReportMonitorRequest(); + if (monitor instanceof SourceMonitor) { + SourceMonitor sourceMonitor = (SourceMonitor) monitor; + request.setTaskID(sourceMonitor.getTaskId()); + request.setJobID(sourceMonitor.getJobId()); + request.setAddress(sourceMonitor.getIp()); + request.setConnectorStage(sourceMonitor.getConnectorStage()); + request.setTotalReqNum(sourceMonitor.getTotalRecordNum().longValue()); + request.setTotalTimeCost(sourceMonitor.getTotalTimeCost().longValue()); + request.setMaxTimeCost(sourceMonitor.getMaxTimeCost().longValue()); + request.setAvgTimeCost(sourceMonitor.getAverageTime()); + request.setTps(sourceMonitor.getTps()); + } else if (monitor instanceof SinkMonitor) { + SinkMonitor sinkMonitor = (SinkMonitor) monitor; + request.setTaskID(sinkMonitor.getTaskId()); + request.setJobID(sinkMonitor.getJobId()); + request.setAddress(sinkMonitor.getIp()); + request.setConnectorStage(sinkMonitor.getConnectorStage()); + request.setTotalReqNum(sinkMonitor.getTotalRecordNum().longValue()); + request.setTotalTimeCost(sinkMonitor.getTotalTimeCost().longValue()); + request.setMaxTimeCost(sinkMonitor.getMaxTimeCost().longValue()); + request.setAvgTimeCost(sinkMonitor.getAverageTime()); + request.setTps(sinkMonitor.getTps()); + } else { + throw new IllegalArgumentException("Unsupported monitor: " + monitor); + } + + Metadata metadata = Metadata.newBuilder() + .setType(ReportMonitorRequest.class.getSimpleName()) + .build(); + Payload payload = Payload.newBuilder() + .setMetadata(metadata) + .setBody(Any.newBuilder().setValue(UnsafeByteOperations.unsafeWrap(Objects.requireNonNull(JsonUtils.toJSONBytes(request)))) + .build()) + .build(); + requestObserver.onNext(payload); + } + + public void stop() { + scheduler.shutdown(); + if (requestObserver != null) { + requestObserver.onCompleted(); + } + } + +} diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/SinkMonitor.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/SinkMonitor.java new file mode 100644 index 0000000000..b27b44da7c --- /dev/null +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/SinkMonitor.java @@ -0,0 +1,52 @@ +/* + * 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.runtime.service.monitor; + +import org.apache.eventmesh.common.enums.ConnectorStage; +import org.apache.eventmesh.openconnect.api.monitor.AbstractConnectorMonitor; + +import lombok.Getter; +import lombok.Setter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@Getter +@Setter +public class SinkMonitor extends AbstractConnectorMonitor { + + private String connectorStage = ConnectorStage.SINK.name(); + + public SinkMonitor(String taskId, String jobId, String ip) { + super(taskId, jobId, ip); + } + + @Override + public void recordProcess(long timeCost) { + super.recordProcess(timeCost); + } + + @Override + public void recordProcess(int recordCount, long timeCost) { + super.recordProcess(recordCount, timeCost); + } + + @Override + public void printMetrics() { + super.printMetrics(); + } +} diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/SourceMonitor.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/SourceMonitor.java new file mode 100644 index 0000000000..3895c8df14 --- /dev/null +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/monitor/SourceMonitor.java @@ -0,0 +1,47 @@ +/* + * 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.runtime.service.monitor; + +import org.apache.eventmesh.common.enums.ConnectorStage; +import org.apache.eventmesh.openconnect.api.monitor.AbstractConnectorMonitor; + +import lombok.Getter; +import lombok.Setter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@Getter +@Setter +public class SourceMonitor extends AbstractConnectorMonitor { + + private String connectorStage = ConnectorStage.SOURCE.name(); + + public SourceMonitor(String taskId, String jobId, String ip) { + super(taskId, jobId, ip); + } + + @Override + public void recordProcess(int recordCount, long timeCost) { + super.recordProcess(recordCount, timeCost); + } + + @Override + public void printMetrics() { + super.printMetrics(); + } +} diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/status/StatusService.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/status/StatusService.java new file mode 100644 index 0000000000..e40686f575 --- /dev/null +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/status/StatusService.java @@ -0,0 +1,94 @@ +/* + * 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.runtime.service.status; + +import org.apache.eventmesh.common.protocol.grpc.adminserver.AdminServiceGrpc; +import org.apache.eventmesh.common.protocol.grpc.adminserver.Metadata; +import org.apache.eventmesh.common.protocol.grpc.adminserver.Payload; +import org.apache.eventmesh.common.remote.JobState; +import org.apache.eventmesh.common.remote.request.ReportJobRequest; +import org.apache.eventmesh.common.utils.IPUtils; +import org.apache.eventmesh.common.utils.JsonUtils; + +import java.util.Objects; + +import io.grpc.stub.StreamObserver; + +import com.google.protobuf.Any; +import com.google.protobuf.UnsafeByteOperations; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class StatusService { + + private StreamObserver requestObserver; + + private StreamObserver responseObserver; + + private AdminServiceGrpc.AdminServiceStub adminServiceStub; + + private AdminServiceGrpc.AdminServiceBlockingStub adminServiceBlockingStub; + + + public StatusService(AdminServiceGrpc.AdminServiceStub adminServiceStub, AdminServiceGrpc.AdminServiceBlockingStub adminServiceBlockingStub) { + this.adminServiceStub = adminServiceStub; + this.adminServiceBlockingStub = adminServiceBlockingStub; + + responseObserver = new StreamObserver() { + @Override + public void onNext(Payload response) { + log.debug("health service receive message: {}|{} ", response.getMetadata(), response.getBody()); + } + + @Override + public void onError(Throwable t) { + log.error("health service receive error message: {}", t.getMessage()); + } + + @Override + public void onCompleted() { + log.info("health service finished receive message and completed"); + } + }; + requestObserver = this.adminServiceStub.invokeBiStream(responseObserver); + } + + public void reportJobStatus(String jobId, JobState jobState) { + ReportJobRequest reportJobRequest = new ReportJobRequest(); + reportJobRequest.setJobID(jobId); + reportJobRequest.setState(jobState); + reportJobRequest.setAddress(IPUtils.getLocalAddress()); + Metadata metadata = Metadata.newBuilder() + .setType(ReportJobRequest.class.getSimpleName()) + .build(); + Payload payload = Payload.newBuilder() + .setMetadata(metadata) + .setBody(Any.newBuilder().setValue(UnsafeByteOperations.unsafeWrap(Objects.requireNonNull(JsonUtils.toJSONBytes(reportJobRequest)))) + .build()) + .build(); + log.info("report job state request: {}", JsonUtils.toJSONString(reportJobRequest)); + requestObserver.onNext(payload); + } + + public void stop() { + if (requestObserver != null) { + requestObserver.onCompleted(); + } + } +} diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/verify/VerifyService.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/verify/VerifyService.java new file mode 100644 index 0000000000..959a2358cd --- /dev/null +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/verify/VerifyService.java @@ -0,0 +1,138 @@ +/* + * 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.runtime.service.verify; + +import org.apache.eventmesh.common.enums.ConnectorStage; +import org.apache.eventmesh.common.protocol.grpc.adminserver.AdminServiceGrpc; +import org.apache.eventmesh.common.protocol.grpc.adminserver.Metadata; +import org.apache.eventmesh.common.protocol.grpc.adminserver.Payload; +import org.apache.eventmesh.common.remote.request.ReportVerifyRequest; +import org.apache.eventmesh.common.utils.IPUtils; +import org.apache.eventmesh.common.utils.JsonUtils; +import org.apache.eventmesh.openconnect.offsetmgmt.api.data.ConnectRecord; +import org.apache.eventmesh.runtime.connector.ConnectorRuntimeConfig; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.Arrays; +import java.util.Objects; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import io.grpc.stub.StreamObserver; + +import com.google.protobuf.Any; +import com.google.protobuf.UnsafeByteOperations; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class VerifyService { + + private final ExecutorService reportVerifyExecutor; + + private StreamObserver requestObserver; + + private StreamObserver responseObserver; + + private AdminServiceGrpc.AdminServiceStub adminServiceStub; + + private AdminServiceGrpc.AdminServiceBlockingStub adminServiceBlockingStub; + + private ConnectorRuntimeConfig connectorRuntimeConfig; + + + public VerifyService(AdminServiceGrpc.AdminServiceStub adminServiceStub, AdminServiceGrpc.AdminServiceBlockingStub adminServiceBlockingStub, + ConnectorRuntimeConfig connectorRuntimeConfig) { + this.adminServiceStub = adminServiceStub; + this.adminServiceBlockingStub = adminServiceBlockingStub; + this.connectorRuntimeConfig = connectorRuntimeConfig; + + this.reportVerifyExecutor = Executors.newSingleThreadExecutor(); + + responseObserver = new StreamObserver() { + @Override + public void onNext(Payload response) { + log.debug("verify service receive message: {}|{} ", response.getMetadata(), response.getBody()); + } + + @Override + public void onError(Throwable t) { + log.error("verify service receive error message: {}", t.getMessage()); + } + + @Override + public void onCompleted() { + log.info("verify service finished receive message and completed"); + } + }; + requestObserver = this.adminServiceStub.invokeBiStream(responseObserver); + } + + public void reportVerifyRequest(ConnectRecord record, ConnectorStage connectorStage) { + reportVerifyExecutor.submit(() -> { + try { + byte[] data = (byte[])record.getData(); + // use record data + recordUniqueId for md5 + String md5Str = md5(Arrays.toString(data) + record.getExtension("recordUniqueId")); + ReportVerifyRequest reportVerifyRequest = new ReportVerifyRequest(); + reportVerifyRequest.setTaskID(connectorRuntimeConfig.getTaskID()); + reportVerifyRequest.setJobID(connectorRuntimeConfig.getJobID()); + reportVerifyRequest.setRecordID(record.getExtension("recordUniqueId")); + 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); + } catch (Exception e) { + log.error("Failed to report verify request", e); + } + }); + } + + 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 void stop() { + reportVerifyExecutor.shutdown(); + if (requestObserver != null) { + requestObserver.onCompleted(); + } + } + +} diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/util/RuntimeUtils.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/util/RuntimeUtils.java index e389357d93..844a9638a3 100644 --- a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/util/RuntimeUtils.java +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/util/RuntimeUtils.java @@ -17,5 +17,18 @@ package org.apache.eventmesh.runtime.util; +import java.util.Random; + public class RuntimeUtils { + + public static String getRandomAdminServerAddr(String adminServerAddrList) { + String[] addresses = adminServerAddrList.split(";"); + if (addresses.length == 0) { + throw new IllegalArgumentException("Admin server address list is empty"); + } + Random random = new Random(); + int randomIndex = random.nextInt(addresses.length); + return addresses[randomIndex]; + } + } From f7781f5e32e2263d8a0e2898b63c64d388f86135 Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Sun, 8 Dec 2024 18:56:35 +0800 Subject: [PATCH 2/5] fix checkStyle error --- .../runtime/service/verify/VerifyService.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/verify/VerifyService.java b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/verify/VerifyService.java index 959a2358cd..8bcb72199c 100644 --- a/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/verify/VerifyService.java +++ b/eventmesh-runtime-v2/src/main/java/org/apache/eventmesh/runtime/service/verify/VerifyService.java @@ -87,7 +87,7 @@ public void onCompleted() { public void reportVerifyRequest(ConnectRecord record, ConnectorStage connectorStage) { reportVerifyExecutor.submit(() -> { try { - byte[] data = (byte[])record.getData(); + byte[] data = (byte[]) record.getData(); // use record data + recordUniqueId for md5 String md5Str = md5(Arrays.toString(data) + record.getExtension("recordUniqueId")); ReportVerifyRequest reportVerifyRequest = new ReportVerifyRequest(); @@ -96,17 +96,17 @@ public void reportVerifyRequest(ConnectRecord record, ConnectorStage connectorSt reportVerifyRequest.setRecordID(record.getExtension("recordUniqueId")); reportVerifyRequest.setRecordSig(md5Str); reportVerifyRequest.setConnectorName( - IPUtils.getLocalAddress() + "_" + connectorRuntimeConfig.getJobID() + "_" + connectorRuntimeConfig.getRegion()); + 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(); + .setBody( + Any.newBuilder().setValue(UnsafeByteOperations.unsafeWrap(Objects.requireNonNull(JsonUtils.toJSONBytes(reportVerifyRequest)))) + .build()) + .build(); requestObserver.onNext(request); } catch (Exception e) { log.error("Failed to report verify request", e); From 05d90709d87764a627ea083e0d81c6a2a92364af Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Mon, 9 Dec 2024 16:20:17 +0800 Subject: [PATCH 3/5] [ISSUE #5139] update canal connector module --- .../rdb/canal/CanalSinkFullConfig.java | 1 + .../rdb/canal/CanalSinkIncrementConfig.java | 4 +- .../rdb/canal/CanalSourceCheckConfig.java | 38 ++ .../rdb/canal/CanalSourceFullConfig.java | 6 +- .../rdb/canal/CanalSourceIncrementConfig.java | 16 +- .../rdb/canal/JobRdbFullPosition.java | 1 + .../rdb/canal/mysql/MySQLTableDef.java | 4 +- .../datasource/DataSourceDriverType.java | 1 + .../remote/datasource/DataSourceType.java | 1 + .../eventmesh/connector/canal/SqlUtils.java | 4 +- .../SqlBuilderLoadInterceptor.java | 25 +- .../sink/connector/CanalCheckConsumer.java | 540 ++++++++++++++++++ .../sink/connector/CanalFullConsumer.java | 391 +++++++++++++ .../connector/CanalSinkCheckConnector.java | 341 ++--------- .../connector/CanalSinkFullConnector.java | 362 ++---------- .../CanalSinkIncrementConnector.java | 2 +- .../connector/canal/source/EntryParser.java | 19 +- .../source/connector/CanalFullProducer.java | 100 +++- .../connector/CanalSourceCheckConnector.java | 126 ++-- .../connector/CanalSourceFullConnector.java | 22 +- .../CanalSourceIncrementConnector.java | 115 +++- .../position/CanalCheckPositionMgr.java | 250 ++++++++ .../source/position/CanalFullPositionMgr.java | 8 +- .../canal/source/table/RdbTableMgr.java | 82 ++- 24 files changed, 1673 insertions(+), 786 deletions(-) create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceCheckConfig.java create mode 100644 eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalCheckConsumer.java create mode 100644 eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalFullConsumer.java create mode 100644 eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/position/CanalCheckPositionMgr.java diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkFullConfig.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkFullConfig.java index f1d78a65dc..dca16b100c 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkFullConfig.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkFullConfig.java @@ -28,4 +28,5 @@ public class CanalSinkFullConfig extends SinkConfig { private SinkConnectorConfig sinkConnectorConfig; private String zeroDate; + private int parallel = 2; } diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkIncrementConfig.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkIncrementConfig.java index 32112a769b..aeb9d5a0e2 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkIncrementConfig.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSinkIncrementConfig.java @@ -36,9 +36,9 @@ public class CanalSinkIncrementConfig extends CanalSinkConfig { private Integer poolSize = 5; // sync mode: field/row - private SyncMode syncMode; + private SyncMode syncMode = SyncMode.ROW; - private boolean isGTIDMode = true; + private boolean isGTIDMode = false; private boolean isMariaDB = true; diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceCheckConfig.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceCheckConfig.java new file mode 100644 index 0000000000..f326301d7d --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceCheckConfig.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.common.config.connector.rdb.canal; + +import org.apache.eventmesh.common.config.connector.SourceConfig; +import org.apache.eventmesh.common.remote.offset.RecordPosition; + +import java.util.List; + +import lombok.Data; +import lombok.EqualsAndHashCode; + +@Data +@EqualsAndHashCode(callSuper = true) +public class CanalSourceCheckConfig extends SourceConfig { + private SourceConnectorConfig sourceConnectorConfig; + private List startPosition; + private int parallel; + private int flushSize; + private int executePeriod = 3600; + private Integer pagePerSecond = 1; + private Integer recordPerSecond = 100; +} diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceFullConfig.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceFullConfig.java index 15398b303a..53988ca055 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceFullConfig.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceFullConfig.java @@ -30,6 +30,8 @@ public class CanalSourceFullConfig extends SourceConfig { private SourceConnectorConfig sourceConnectorConfig; private List startPosition; - private int parallel; - private int flushSize; + private int parallel = 2; + private int flushSize = 20; + private Integer pagePerSecond = 1; + private Integer recordPerSecond = 100; } diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceIncrementConfig.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceIncrementConfig.java index 94fe007b5f..7f73727140 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceIncrementConfig.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/CanalSourceIncrementConfig.java @@ -32,17 +32,17 @@ public class CanalSourceIncrementConfig extends CanalSourceConfig { private String destination; - private Long canalInstanceId; + private Long canalInstanceId = 1L; - private String desc; + private String desc = "canalSourceInstance"; - private boolean ddlSync = true; + private boolean ddlSync = false; private boolean filterTableError = false; private Long slaveId; - private Short clientId; + private Short clientId = 1; private String serverUUID; @@ -67,19 +67,19 @@ public class CanalSourceIncrementConfig extends CanalSourceConfig { private Boolean enableRemedy = false; // sync mode: field/row - private SyncMode syncMode; + private SyncMode syncMode = SyncMode.ROW; // sync consistency - private SyncConsistency syncConsistency; + private SyncConsistency syncConsistency = SyncConsistency.BASE; // ================================= system parameter // ================================ // Column name of the bidirectional synchronization mark - private String needSyncMarkTableColumnName = "needSync"; + private String needSyncMarkTableColumnName; // Column value of the bidirectional synchronization mark - private String needSyncMarkTableColumnValue = "needSync"; + private String needSyncMarkTableColumnValue; private SourceConnectorConfig sourceConnectorConfig; diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/JobRdbFullPosition.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/JobRdbFullPosition.java index 08f88e1d24..42ba889bbd 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/JobRdbFullPosition.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/JobRdbFullPosition.java @@ -30,6 +30,7 @@ public class JobRdbFullPosition { private String tableName; private String primaryKeyRecords; private long maxCount; + private long handledRecordCount = 0; private boolean finished; private BigDecimal percent; } diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/mysql/MySQLTableDef.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/mysql/MySQLTableDef.java index cdd3652378..4266a96060 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/mysql/MySQLTableDef.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/rdb/canal/mysql/MySQLTableDef.java @@ -19,8 +19,8 @@ import org.apache.eventmesh.common.config.connector.rdb.canal.RdbTableDefinition; +import java.util.List; import java.util.Map; -import java.util.Set; import lombok.Data; import lombok.EqualsAndHashCode; @@ -31,6 +31,6 @@ @Data @EqualsAndHashCode(callSuper = true) public class MySQLTableDef extends RdbTableDefinition { - private Set primaryKeys; + private List primaryKeys; private Map columnDefinitions; } diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSourceDriverType.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSourceDriverType.java index 4429bee5a9..f1c0f54e5f 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSourceDriverType.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSourceDriverType.java @@ -19,6 +19,7 @@ public enum DataSourceDriverType { MYSQL, + MariaDB, REDIS, ROCKETMQ, HTTP; diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSourceType.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSourceType.java index 8c40971e7b..1c14239c3b 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSourceType.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/datasource/DataSourceType.java @@ -27,6 +27,7 @@ @ToString public enum DataSourceType { MYSQL("MySQL", DataSourceDriverType.MYSQL, DataSourceClassify.RDB), + MariaDB("MariaDB", DataSourceDriverType.MariaDB, DataSourceClassify.RDB), REDIS("Redis", DataSourceDriverType.REDIS, DataSourceClassify.CACHE), ROCKETMQ("RocketMQ", DataSourceDriverType.ROCKETMQ, DataSourceClassify.MQ), HTTP("HTTP", DataSourceDriverType.HTTP, DataSourceClassify.TUNNEL); diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/SqlUtils.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/SqlUtils.java index 1008ad1cf3..273f5cde4c 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/SqlUtils.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/SqlUtils.java @@ -780,8 +780,8 @@ public static LocalDateTime toLocalDateTime(Object value) { long nanos = ((Timestamp) value).getNanos(); return Instant.ofEpochMilli(((Timestamp) value).getTime() - (nanos / 1000000)).plusNanos(nanos).atZone(ZoneId.systemDefault()) .toLocalDateTime(); - } else if (value instanceof java.sql.Date) { - return ((java.sql.Date) value).toLocalDate().atTime(0, 0); + } else if (value instanceof Date) { + return ((Date) value).toLocalDate().atTime(0, 0); } else { if (!(value instanceof Time)) { return ((java.util.Date) value).toInstant().atZone(ZoneId.systemDefault()).toLocalDateTime(); 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 7d83bd4f3f..1d7bd35b94 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 @@ -64,6 +64,7 @@ public boolean before(CanalSinkIncrementConfig sinkConfig, CanalConnectRecord re String[] keyColumns = null; String[] otherColumns = null; if (existOldKeys) { + // update table xxx set pk = newPK where pk = oldPk keyColumns = buildColumnNames(record.getOldKeys()); otherColumns = buildColumnNames(record.getUpdatedColumns(), record.getKeys()); } else { @@ -71,17 +72,19 @@ public boolean before(CanalSinkIncrementConfig sinkConfig, CanalConnectRecord re otherColumns = buildColumnNames(record.getUpdatedColumns()); } - if (rowMode && !existOldKeys) { - sql = sqlTemplate.getMergeSql(schemaName, - record.getTableName(), - keyColumns, - otherColumns, - new String[] {}, - true, - shardColumns); - } else { - sql = sqlTemplate.getUpdateSql(schemaName, record.getTableName(), keyColumns, otherColumns, true, shardColumns); - } + // not support the column default not null for merge sql + // if (rowMode && !existOldKeys) { + // sql = sqlTemplate.getMergeSql(schemaName, + // record.getTableName(), + // keyColumns, + // otherColumns, + // new String[] {}, + // true, + // shardColumns); + // } else { + // sql = sqlTemplate.getUpdateSql(schemaName, record.getTableName(), keyColumns, otherColumns, true, shardColumns); + // } + sql = sqlTemplate.getUpdateSql(schemaName, record.getTableName(), keyColumns, otherColumns, true, shardColumns); } else if (type.isDelete()) { sql = sqlTemplate.getDeleteSql(schemaName, record.getTableName(), diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalCheckConsumer.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalCheckConsumer.java new file mode 100644 index 0000000000..fb9a33b49f --- /dev/null +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalCheckConsumer.java @@ -0,0 +1,540 @@ +/* + * 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.connector; + +import org.apache.eventmesh.common.config.connector.rdb.canal.CanalMySQLType; +import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSinkFullConfig; +import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.Constants; +import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.MySQLColumnDef; +import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.MySQLTableDef; +import org.apache.eventmesh.common.remote.offset.canal.CanalFullRecordOffset; +import org.apache.eventmesh.common.utils.JsonUtils; +import org.apache.eventmesh.connector.canal.DatabaseConnection; +import org.apache.eventmesh.connector.canal.SqlUtils; +import org.apache.eventmesh.connector.canal.source.table.RdbTableMgr; +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.lang3.StringUtils; + +import java.math.BigDecimal; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; +import java.text.MessageFormat; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.LockSupport; + +import com.alibaba.druid.pool.DruidPooledConnection; +import com.fasterxml.jackson.core.type.TypeReference; + +import lombok.extern.slf4j.Slf4j; + + +@Slf4j +public class CanalCheckConsumer { + private BlockingQueue> queue; + private RdbTableMgr tableMgr; + private CanalSinkFullConfig config; + private final DateTimeFormatter dataTimePattern = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS"); + + + public CanalCheckConsumer(BlockingQueue> queue, RdbTableMgr tableMgr, CanalSinkFullConfig config) { + this.config = config; + this.queue = queue; + this.tableMgr = tableMgr; + } + + + public void start(AtomicBoolean flag) { + while (flag.get()) { + List sinkRecords = null; + try { + sinkRecords = queue.poll(2, TimeUnit.SECONDS); + if (sinkRecords == null || sinkRecords.isEmpty()) { + continue; + } + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + ConnectRecord record = sinkRecords.get(0); + Map dataMap = + JsonUtils.parseTypeReferenceObject((byte[]) record.getData(), new TypeReference>() { + }); + + List> sourceRows = JsonUtils.parseObject(dataMap.get("data").toString(), List.class); + + if (sourceRows == null || sourceRows.isEmpty()) { + if (log.isDebugEnabled()) { + log.debug("[{}] got rows data is none", this.getClass()); + } + return; + } + CanalFullRecordOffset offset = JsonUtils.parseObject(dataMap.get("offset").toString(), CanalFullRecordOffset.class); + if (offset == null || offset.getPosition() == null) { + if (log.isDebugEnabled()) { + log.debug("[{}] got canal full offset is none", this.getClass()); + } + return; + } + + MySQLTableDef tableDefinition = (MySQLTableDef) tableMgr.getTable(offset.getPosition().getSchema(), offset.getPosition().getTableName()); + if (tableDefinition == null) { + log.warn("target schema [{}] table [{}] is not exists", offset.getPosition().getSchema(), offset.getPosition().getTableName()); + return; + } + + String sql = genTargetPkInSql(tableDefinition, sourceRows.size(), Constants.MySQLQuot, Constants.MySQLQuot, "*"); + DruidPooledConnection connection = null; + PreparedStatement statement = null; + try { + connection = DatabaseConnection.sinkDataSource.getConnection(); + statement = + connection.prepareStatement(sql); + setPrepareParams(statement, sourceRows, tableDefinition); + log.debug("select sql {}", statement.toString()); + ResultSet resultSet = statement.executeQuery(); + List> targetRows = new LinkedList<>(); + while (resultSet.next()) { + Map columnValues = new LinkedHashMap<>(); + for (Map.Entry col : + tableDefinition.getColumnDefinitions().entrySet()) { + columnValues.put(col.getKey(), readColumn(resultSet, col.getKey(), + col.getValue().getType())); + } + targetRows.add(columnValues); + } + compareData(sourceRows, targetRows, tableDefinition); + record.getCallback().onSuccess(convertToSendResult(record)); + } catch (SQLException e) { + log.warn("check sink process schema [{}] table [{}] connector check fail", tableDefinition.getSchemaName(), + tableDefinition.getTableName(), + e); + LockSupport.parkNanos(3000 * 1000L); + record.getCallback().onException(buildSendExceptionContext(record, e)); + } catch (Exception e) { + log.error("check sink process schema [{}] table [{}] catch unknown exception", tableDefinition.getSchemaName(), + tableDefinition.getTableName(), e); + record.getCallback().onException(buildSendExceptionContext(record, e)); + } finally { + if (statement != null) { + try { + statement.close(); + } catch (SQLException e) { + log.error("close prepare statement fail", e); + } + } + + if (connection != null) { + try { + connection.close(); + } catch (SQLException e) { + log.error("close db connection fail", e); + } + } + } + } + } + + private void compareData(List> sourceRows, List> targetRows, MySQLTableDef tableDefinition) { + List> differenceSource = new ArrayList<>(sourceRows); + List> differenceTarget = new ArrayList<>(targetRows); + // Find common elements and remove from difference lists + for (Map source : sourceRows) { + for (Map target : targetRows) { + if (source.equals(target)) { + differenceSource.remove(source); + differenceTarget.remove(target); + break; + } + } + } + if (!differenceSource.isEmpty()) { + log.error("source rows is not equals target rows, source rows are [{}]", differenceSource); + } + + if (!differenceTarget.isEmpty()) { + log.error("source rows is not equals target rows, target rows are [{}]", differenceTarget); + } + } + + private void setPrepareParams(PreparedStatement preparedStatement, List> rows, MySQLTableDef tableDef) throws Exception { + List cols = new ArrayList<>(tableDef.getColumnDefinitions().values()); + int index = 0; + for (Map col : rows) { + for (MySQLColumnDef mySQLColumnDef : cols) { + if (tableDef.getPrimaryKeys().contains(mySQLColumnDef.getName())) { + index++; + writeColumn(preparedStatement, index, mySQLColumnDef, col.get(mySQLColumnDef.getName())); + } + } + } + } + + public Object readColumn(ResultSet rs, String colName, CanalMySQLType colType) throws Exception { + switch (colType) { + case TINYINT: + case SMALLINT: + case MEDIUMINT: + case INT: + Long valueLong = rs.getLong(colName); + if (rs.wasNull()) { + return null; + } + if (valueLong.compareTo((long) Integer.MAX_VALUE) > 0) { + return valueLong; + } + return valueLong.intValue(); + case BIGINT: + String v = rs.getString(colName); + if (v == null) { + return null; + } + BigDecimal valueBigInt = new BigDecimal(v); + if (valueBigInt.compareTo(BigDecimal.valueOf(Long.MAX_VALUE)) > 0) { + return valueBigInt; + } + return valueBigInt.longValue(); + case FLOAT: + case DOUBLE: + case DECIMAL: + return rs.getBigDecimal(colName); + case DATE: + return rs.getObject(colName, LocalDate.class).toString(); + case TIME: + return rs.getObject(colName, LocalTime.class).toString(); + case DATETIME: + case TIMESTAMP: + return rs.getObject(colName, LocalDateTime.class).toString(); + case YEAR: + int year = rs.getInt(colName); + if (rs.wasNull()) { + return null; + } + return year; + case CHAR: + case VARCHAR: + case TINYTEXT: + case TEXT: + case MEDIUMTEXT: + case LONGTEXT: + case ENUM: + case SET: + case JSON: + return rs.getString(colName); + case BIT: + case BINARY: + case VARBINARY: + case TINYBLOB: + case BLOB: + case MEDIUMBLOB: + case LONGBLOB: + return rs.getBytes(colName); + case GEOMETRY: + case GEOMETRY_COLLECTION: + case GEOM_COLLECTION: + case POINT: + case LINESTRING: + case POLYGON: + case MULTIPOINT: + case MULTILINESTRING: + case MULTIPOLYGON: + byte[] geo = rs.getBytes(colName); + if (geo == null) { + return null; + } + return SqlUtils.toGeometry(geo); + default: + return rs.getObject(colName); + } + } + + public void writeColumn(PreparedStatement ps, int index, MySQLColumnDef colType, Object value) throws Exception { + if (colType == null) { + String colVal = null; + if (value != null) { + colVal = value.toString(); + } + if (colVal == null) { + ps.setNull(index, Types.VARCHAR); + } else { + ps.setString(index, colVal); + } + } else if (value == null) { + ps.setNull(index, colType.getJdbcType().getVendorTypeNumber()); + } else { + switch (colType.getType()) { + case TINYINT: + case SMALLINT: + case MEDIUMINT: + case INT: + Long longValue = SqlUtils.toLong(value); + if (longValue == null) { + ps.setNull(index, 4); + return; + } else { + ps.setLong(index, longValue); + return; + } + case BIGINT: + case DECIMAL: + BigDecimal bigDecimalValue = SqlUtils.toBigDecimal(value); + if (bigDecimalValue == null) { + ps.setNull(index, 3); + return; + } else { + ps.setBigDecimal(index, bigDecimalValue); + return; + } + case FLOAT: + case DOUBLE: + Double doubleValue = SqlUtils.toDouble(value); + if (doubleValue == null) { + ps.setNull(index, 8); + } else { + ps.setDouble(index, doubleValue); + } + return; + case DATE: + case DATETIME: + case TIMESTAMP: + LocalDateTime dateValue = null; + if (!SqlUtils.isZeroTime(value)) { + try { + dateValue = SqlUtils.toLocalDateTime(value); + } catch (Exception e) { + ps.setString(index, SqlUtils.convertToString(value)); + return; + } + } else if (StringUtils.isNotBlank(config.getZeroDate())) { + dateValue = SqlUtils.toLocalDateTime(config.getZeroDate()); + } else { + ps.setObject(index, value); + return; + } + if (dateValue == null) { + ps.setNull(index, Types.TIMESTAMP); + } else { + ps.setString(index, dataTimePattern.format(dateValue)); + } + return; + case TIME: + String timeValue = SqlUtils.toMySqlTime(value); + if (StringUtils.isBlank(timeValue)) { + ps.setNull(index, 12); + return; + } else { + ps.setString(index, timeValue); + return; + } + case YEAR: + LocalDateTime yearValue = null; + if (!SqlUtils.isZeroTime(value)) { + yearValue = SqlUtils.toLocalDateTime(value); + } else if (StringUtils.isNotBlank(config.getZeroDate())) { + yearValue = SqlUtils.toLocalDateTime(config.getZeroDate()); + } else { + ps.setInt(index, 0); + return; + } + if (yearValue == null) { + ps.setNull(index, 4); + } else { + ps.setInt(index, yearValue.getYear()); + } + return; + case CHAR: + case VARCHAR: + case TINYTEXT: + case TEXT: + case MEDIUMTEXT: + case LONGTEXT: + case ENUM: + case SET: + String strValue = value.toString(); + if (strValue == null) { + ps.setNull(index, Types.VARCHAR); + return; + } else { + ps.setString(index, strValue); + return; + } + case JSON: + String jsonValue = value.toString(); + if (jsonValue == null) { + ps.setNull(index, Types.VARCHAR); + } else { + ps.setString(index, jsonValue); + } + return; + case BIT: + if (value instanceof Boolean) { + byte[] arrayBoolean = new byte[1]; + arrayBoolean[0] = (byte) (Boolean.TRUE.equals(value) ? 1 : 0); + ps.setBytes(index, arrayBoolean); + return; + } else if (value instanceof Number) { + ps.setBytes(index, SqlUtils.numberToBinaryArray((Number) value)); + return; + } else if ((value instanceof byte[]) || value.toString().startsWith("0x") || value.toString().startsWith("0X")) { + byte[] arrayBoolean = SqlUtils.toBytes(value); + if (arrayBoolean == null || arrayBoolean.length == 0) { + ps.setNull(index, Types.BIT); + return; + } else { + ps.setBytes(index, arrayBoolean); + return; + } + } else { + ps.setBytes(index, SqlUtils.numberToBinaryArray(SqlUtils.toInt(value))); + return; + } + case BINARY: + case VARBINARY: + case TINYBLOB: + case BLOB: + case MEDIUMBLOB: + case LONGBLOB: + byte[] binaryValue = SqlUtils.toBytes(value); + if (binaryValue == null) { + ps.setNull(index, Types.BINARY); + return; + } else { + ps.setBytes(index, binaryValue); + return; + } + case GEOMETRY: + case GEOMETRY_COLLECTION: + case GEOM_COLLECTION: + case POINT: + case LINESTRING: + case POLYGON: + case MULTIPOINT: + case MULTILINESTRING: + case MULTIPOLYGON: + String geoValue = SqlUtils.toGeometry(value); + if (geoValue == null) { + ps.setNull(index, Types.VARCHAR); + return; + } + ps.setString(index, geoValue); + return; + default: + throw new UnsupportedOperationException("columnType '" + colType + "' Unsupported."); + } + } + } + + public String genTargetPkInSql(MySQLTableDef def, int pkGroupSize, String leftQuote, String rightQuote, String selectEleStr) { + List pkCols = def.getPrimaryKeys(); + if (pkCols == null || pkCols.isEmpty()) { + throw new IllegalArgumentException("unsupported pk is empty table check."); + } else if (pkCols.size() == 1) { + return genSinglePkInSql(def, pkGroupSize, leftQuote, rightQuote, selectEleStr); + } else { + return genMultiPkInSql(def, pkGroupSize, leftQuote, rightQuote, selectEleStr); + } + } + + public String genSinglePkInSql(MySQLTableDef def, int pkGroupSize, String leftQuote, String rightQuote, String selectEleStr) { + return MessageFormat.format(genFetchSqlFormat(leftQuote, rightQuote, selectEleStr), def.getSchemaName(), def.getTableName(), + leftQuote + def.getPrimaryKeys().get(0) + rightQuote, genSinglePkPlaceHolderStr(pkGroupSize)); + } + + public String genMultiPkInSql(MySQLTableDef def, int pkGroupSize, String leftQuote, String rightQuote, String selectEleStr) { + String fetchSqlFormat = genFetchSqlFormat(leftQuote, rightQuote, selectEleStr); + List pkCols = def.getPrimaryKeys(); + StringBuilder pksBuilder = new StringBuilder("("); + for (int i = 0; i < pkCols.size(); i++) { + if (i != 0) { + pksBuilder.append(","); + } + pksBuilder.append(leftQuote).append(pkCols.get(i)).append(rightQuote); + } + pksBuilder.append(")"); + return MessageFormat.format(fetchSqlFormat, def.getSchemaName(), def.getTableName(), pksBuilder.toString(), + genMultiPkPlaceHolderStr(pkGroupSize, pkCols.size())); + } + + public String genFetchSqlFormat(String leftQuote, String rightQuote, String selectEleStr) { + return "select " + selectEleStr + " from " + leftQuote + "{0}" + rightQuote + "." + leftQuote + "{1}" + rightQuote + " where {2} in ({3})"; + } + + public String genSinglePkPlaceHolderStr(int valueSize) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < valueSize; i++) { + if (i != 0) { + sb.append(","); + } + sb.append("?"); + } + return sb.toString(); + } + + public String genMultiPkPlaceHolderStr(int valueSize, int sizePerGroup) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < valueSize; i++) { + if (i != 0) { + sb.append(","); + } + sb.append("("); + for (int j = 0; j < sizePerGroup; j++) { + if (j != 0) { + sb.append(","); + } + sb.append("?"); + } + sb.append(")"); + } + return sb.toString(); + } + + + + private SendExceptionContext buildSendExceptionContext(ConnectRecord record, Throwable e) { + SendExceptionContext sendExceptionContext = new SendExceptionContext(); + sendExceptionContext.setMessageId(record.getRecordId()); + sendExceptionContext.setCause(e); + if (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 (StringUtils.isNotEmpty(record.getExtension("topic"))) { + result.setTopic(record.getExtension("topic")); + } + return result; + } +} diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalFullConsumer.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalFullConsumer.java new file mode 100644 index 0000000000..939d1101aa --- /dev/null +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalFullConsumer.java @@ -0,0 +1,391 @@ +/* + * 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.connector; + +import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSinkFullConfig; +import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.Constants; +import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.MySQLColumnDef; +import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.MySQLTableDef; +import org.apache.eventmesh.common.remote.offset.canal.CanalFullRecordOffset; +import org.apache.eventmesh.common.utils.JsonUtils; +import org.apache.eventmesh.connector.canal.DatabaseConnection; +import org.apache.eventmesh.connector.canal.SqlUtils; +import org.apache.eventmesh.connector.canal.source.table.RdbTableMgr; +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.lang3.StringUtils; + +import java.math.BigDecimal; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.sql.Types; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.LockSupport; + +import com.alibaba.druid.pool.DruidPooledConnection; +import com.fasterxml.jackson.core.type.TypeReference; + +import lombok.extern.slf4j.Slf4j; + + +@Slf4j +public class CanalFullConsumer { + private BlockingQueue> queue; + private RdbTableMgr tableMgr; + private CanalSinkFullConfig config; + private final DateTimeFormatter dataTimePattern = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS"); + + + public CanalFullConsumer(BlockingQueue> queue, RdbTableMgr tableMgr, CanalSinkFullConfig config) { + this.config = config; + this.queue = queue; + this.tableMgr = tableMgr; + } + + + public void start(AtomicBoolean flag) { + while (flag.get()) { + List sinkRecords = null; + try { + sinkRecords = queue.poll(2, TimeUnit.SECONDS); + if (sinkRecords == null || sinkRecords.isEmpty()) { + continue; + } + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + ConnectRecord record = sinkRecords.get(0); + Map dataMap = + JsonUtils.parseTypeReferenceObject((byte[]) record.getData(), new TypeReference>() { + }); + + List> rows = JsonUtils.parseObject(dataMap.get("data").toString(), List.class); + + if (rows == null || rows.isEmpty()) { + if (log.isDebugEnabled()) { + log.debug("[{}] got rows data is none", this.getClass()); + } + return; + } + CanalFullRecordOffset offset = JsonUtils.parseObject(dataMap.get("offset").toString(), CanalFullRecordOffset.class); + if (offset == null || offset.getPosition() == null) { + if (log.isDebugEnabled()) { + log.debug("[{}] got canal full offset is none", this.getClass()); + } + return; + } + + MySQLTableDef tableDefinition = (MySQLTableDef) tableMgr.getTable(offset.getPosition().getSchema(), offset.getPosition().getTableName()); + if (tableDefinition == null) { + log.warn("target schema [{}] table [{}] is not exists", offset.getPosition().getSchema(), offset.getPosition().getTableName()); + return; + } + List cols = new ArrayList<>(tableDefinition.getColumnDefinitions().values()); + String sql = generateInsertPrepareSql(offset.getPosition().getSchema(), offset.getPosition().getTableName(), + cols); + DruidPooledConnection connection = null; + PreparedStatement statement = null; + try { + connection = DatabaseConnection.sinkDataSource.getConnection(); + statement = + connection.prepareStatement(sql); + for (Map col : rows) { + setPrepareParams(statement, col, cols); + log.debug("insert sql {}", statement.toString()); + statement.addBatch(); + } + statement.executeBatch(); + connection.commit(); + log.info("execute batch insert sql size: {}", rows.size()); + record.getCallback().onSuccess(convertToSendResult(record)); + } catch (SQLException e) { + log.warn("full sink process schema [{}] table [{}] connector write fail", tableDefinition.getSchemaName(), + tableDefinition.getTableName(), + e); + LockSupport.parkNanos(3000 * 1000L); + record.getCallback().onException(buildSendExceptionContext(record, e)); + } catch (Exception e) { + log.error("full sink process schema [{}] table [{}] catch unknown exception", tableDefinition.getSchemaName(), + tableDefinition.getTableName(), e); + record.getCallback().onException(buildSendExceptionContext(record, e)); + try { + if (connection != null && !connection.isClosed()) { + connection.rollback(); + } + } catch (SQLException rollback) { + log.warn("full sink process schema [{}] table [{}] rollback fail", tableDefinition.getSchemaName(), + tableDefinition.getTableName(), e); + } + } finally { + if (statement != null) { + try { + statement.close(); + } catch (SQLException e) { + log.error("close prepare statement fail", e); + } + } + + if (connection != null) { + try { + connection.close(); + } catch (SQLException e) { + log.error("close db connection fail", e); + } + } + } + } + } + + + private SendExceptionContext buildSendExceptionContext(ConnectRecord record, Throwable e) { + SendExceptionContext sendExceptionContext = new SendExceptionContext(); + sendExceptionContext.setMessageId(record.getRecordId()); + sendExceptionContext.setCause(e); + if (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 (StringUtils.isNotEmpty(record.getExtension("topic"))) { + result.setTopic(record.getExtension("topic")); + } + return result; + } + + private void setPrepareParams(PreparedStatement preparedStatement, Map col, List columnDefs) throws Exception { + for (int i = 0; i < columnDefs.size(); i++) { + writeColumn(preparedStatement, i + 1, columnDefs.get(i), col.get(columnDefs.get(i).getName())); + } + } + + public void writeColumn(PreparedStatement ps, int index, MySQLColumnDef colType, Object value) throws Exception { + if (colType == null) { + String colVal = null; + if (value != null) { + colVal = value.toString(); + } + if (colVal == null) { + ps.setNull(index, Types.VARCHAR); + } else { + ps.setString(index, colVal); + } + } else if (value == null) { + ps.setNull(index, colType.getJdbcType().getVendorTypeNumber()); + } else { + switch (colType.getType()) { + case TINYINT: + case SMALLINT: + case MEDIUMINT: + case INT: + Long longValue = SqlUtils.toLong(value); + if (longValue == null) { + ps.setNull(index, 4); + return; + } else { + ps.setLong(index, longValue); + return; + } + case BIGINT: + case DECIMAL: + BigDecimal bigDecimalValue = SqlUtils.toBigDecimal(value); + if (bigDecimalValue == null) { + ps.setNull(index, 3); + return; + } else { + ps.setBigDecimal(index, bigDecimalValue); + return; + } + case FLOAT: + case DOUBLE: + Double doubleValue = SqlUtils.toDouble(value); + if (doubleValue == null) { + ps.setNull(index, 8); + } else { + ps.setDouble(index, doubleValue); + } + return; + case DATE: + case DATETIME: + case TIMESTAMP: + LocalDateTime dateValue = null; + if (!SqlUtils.isZeroTime(value)) { + try { + dateValue = SqlUtils.toLocalDateTime(value); + } catch (Exception e) { + ps.setString(index, SqlUtils.convertToString(value)); + return; + } + } else if (StringUtils.isNotBlank(config.getZeroDate())) { + dateValue = SqlUtils.toLocalDateTime(config.getZeroDate()); + } else { + ps.setObject(index, value); + return; + } + if (dateValue == null) { + ps.setNull(index, Types.TIMESTAMP); + } else { + ps.setString(index, dataTimePattern.format(dateValue)); + } + return; + case TIME: + String timeValue = SqlUtils.toMySqlTime(value); + if (StringUtils.isBlank(timeValue)) { + ps.setNull(index, 12); + return; + } else { + ps.setString(index, timeValue); + return; + } + case YEAR: + LocalDateTime yearValue = null; + if (!SqlUtils.isZeroTime(value)) { + yearValue = SqlUtils.toLocalDateTime(value); + } else if (StringUtils.isNotBlank(config.getZeroDate())) { + yearValue = SqlUtils.toLocalDateTime(config.getZeroDate()); + } else { + ps.setInt(index, 0); + return; + } + if (yearValue == null) { + ps.setNull(index, 4); + } else { + ps.setInt(index, yearValue.getYear()); + } + return; + case CHAR: + case VARCHAR: + case TINYTEXT: + case TEXT: + case MEDIUMTEXT: + case LONGTEXT: + case ENUM: + case SET: + String strValue = value.toString(); + if (strValue == null) { + ps.setNull(index, Types.VARCHAR); + return; + } else { + ps.setString(index, strValue); + return; + } + case JSON: + String jsonValue = value.toString(); + if (jsonValue == null) { + ps.setNull(index, Types.VARCHAR); + } else { + ps.setString(index, jsonValue); + } + return; + case BIT: + if (value instanceof Boolean) { + byte[] arrayBoolean = new byte[1]; + arrayBoolean[0] = (byte) (Boolean.TRUE.equals(value) ? 1 : 0); + ps.setBytes(index, arrayBoolean); + return; + } else if (value instanceof Number) { + ps.setBytes(index, SqlUtils.numberToBinaryArray((Number) value)); + return; + } else if ((value instanceof byte[]) || value.toString().startsWith("0x") || value.toString().startsWith("0X")) { + byte[] arrayBoolean = SqlUtils.toBytes(value); + if (arrayBoolean == null || arrayBoolean.length == 0) { + ps.setNull(index, Types.BIT); + return; + } else { + ps.setBytes(index, arrayBoolean); + return; + } + } else { + ps.setBytes(index, SqlUtils.numberToBinaryArray(SqlUtils.toInt(value))); + return; + } + case BINARY: + case VARBINARY: + case TINYBLOB: + case BLOB: + case MEDIUMBLOB: + case LONGBLOB: + byte[] binaryValue = SqlUtils.toBytes(value); + if (binaryValue == null) { + ps.setNull(index, Types.BINARY); + return; + } else { + ps.setBytes(index, binaryValue); + return; + } + case GEOMETRY: + case GEOMETRY_COLLECTION: + case GEOM_COLLECTION: + case POINT: + case LINESTRING: + case POLYGON: + case MULTIPOINT: + case MULTILINESTRING: + case MULTIPOLYGON: + String geoValue = SqlUtils.toGeometry(value); + if (geoValue == null) { + ps.setNull(index, Types.VARCHAR); + return; + } + ps.setString(index, geoValue); + return; + default: + throw new UnsupportedOperationException("columnType '" + colType + "' Unsupported."); + } + } + } + + private String generateInsertPrepareSql(String schema, String table, List cols) { + StringBuilder builder = new StringBuilder(); + builder.append("INSERT IGNORE INTO "); + builder.append(Constants.MySQLQuot); + builder.append(schema); + builder.append(Constants.MySQLQuot); + builder.append("."); + builder.append(Constants.MySQLQuot); + builder.append(table); + builder.append(Constants.MySQLQuot); + StringBuilder columns = new StringBuilder(); + StringBuilder values = new StringBuilder(); + for (MySQLColumnDef colInfo : cols) { + if (columns.length() > 0) { + columns.append(", "); + values.append(", "); + } + String wrapName = Constants.MySQLQuot + colInfo.getName() + Constants.MySQLQuot; + columns.append(wrapName); + values.append(colInfo.getType() == null ? "?" : colInfo.getType().genPrepareStatement4Insert()); + } + builder.append("(").append(columns).append(")"); + builder.append(" VALUES "); + builder.append("(").append(values).append(")"); + return builder.toString(); + } +} diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkCheckConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkCheckConnector.java index 84e01ca85c..6819c936fd 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkCheckConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkCheckConnector.java @@ -17,44 +17,38 @@ package org.apache.eventmesh.connector.canal.sink.connector; +import org.apache.eventmesh.common.EventMeshThreadFactory; 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.CanalSinkFullConfig; -import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.Constants; -import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.MySQLColumnDef; -import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.MySQLTableDef; import org.apache.eventmesh.common.exception.EventMeshException; -import org.apache.eventmesh.common.remote.offset.canal.CanalFullRecordOffset; import org.apache.eventmesh.connector.canal.DatabaseConnection; -import org.apache.eventmesh.connector.canal.SqlUtils; 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; import org.apache.eventmesh.openconnect.api.sink.Sink; import org.apache.eventmesh.openconnect.offsetmgmt.api.data.ConnectRecord; +import org.apache.eventmesh.openconnect.util.ConfigUtil; -import org.apache.commons.lang3.StringUtils; - -import java.math.BigDecimal; -import java.sql.PreparedStatement; -import java.sql.SQLException; -import java.sql.Types; -import java.time.LocalDateTime; -import java.time.format.DateTimeFormatter; -import java.util.ArrayList; +import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.concurrent.locks.LockSupport; - -import com.alibaba.druid.pool.DruidPooledConnection; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import lombok.extern.slf4j.Slf4j; @Slf4j public class CanalSinkCheckConnector implements Sink, ConnectorCreateService { + private CanalSinkFullConfig config; private RdbTableMgr tableMgr; - private final DateTimeFormatter dataTimePattern = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS"); + private ThreadPoolExecutor executor; + private final BlockingQueue> queue = new LinkedBlockingQueue<>(10000); + private final AtomicBoolean flag = new AtomicBoolean(true); @Override public void start() throws Exception { @@ -63,7 +57,23 @@ public void start() throws Exception { @Override public void stop() throws Exception { - + flag.set(false); + if (!executor.isShutdown()) { + executor.shutdown(); + try { + if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { + log.warn("wait thread pool shutdown timeout, it will shutdown now"); + executor.shutdownNow(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + log.info("shutdown thread pool fail"); + } + } + if (DatabaseConnection.sinkDataSource != null) { + DatabaseConnection.sinkDataSource.close(); + log.info("data source has been closed"); + } } @Override @@ -84,7 +94,8 @@ public void init(Config config) throws Exception { @Override public void init(ConnectorContext connectorContext) throws Exception { - this.config = (CanalSinkFullConfig) ((SinkConnectorContext) connectorContext).getSinkConfig(); + CanalSinkConfig canalSinkConfig = (CanalSinkConfig) ((SinkConnectorContext) connectorContext).getSinkConfig(); + this.config = ConfigUtil.parse(canalSinkConfig.getSinkConfig(), CanalSinkFullConfig.class); init(); } @@ -97,6 +108,14 @@ private void init() { DatabaseConnection.sinkDataSource.setDefaultAutoCommit(false); tableMgr = new RdbTableMgr(this.config.getSinkConnectorConfig(), DatabaseConnection.sinkDataSource); + executor = new ThreadPoolExecutor(config.getParallel(), config.getParallel(), 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(), new EventMeshThreadFactory("canal-sink-check")); + List consumers = new LinkedList<>(); + for (int i = 0; i < config.getParallel(); i++) { + CanalCheckConsumer canalCheckConsumer = new CanalCheckConsumer(queue, tableMgr, config); + consumers.add(canalCheckConsumer); + } + consumers.forEach(c -> executor.execute(() -> c.start(flag))); } @Override @@ -122,285 +141,11 @@ public void put(List sinkRecords) { } return; } - ConnectRecord record = sinkRecords.get(0); - List> data = (List>) record.getData(); - if (data == null || data.isEmpty()) { - if (log.isDebugEnabled()) { - log.debug("[{}] got rows data is none", this.getClass()); - } - return; - } - CanalFullRecordOffset offset = (CanalFullRecordOffset) record.getPosition().getRecordOffset(); - if (offset == null || offset.getPosition() == null) { - if (log.isDebugEnabled()) { - log.debug("[{}] got canal full offset is none", this.getClass()); - } - return; - } - - MySQLTableDef tableDefinition = (MySQLTableDef) tableMgr.getTable(offset.getPosition().getSchema(), offset.getPosition().getTableName()); - if (tableDefinition == null) { - log.warn("target schema [{}] table [{}] is not exists", offset.getPosition().getSchema(), offset.getPosition().getTableName()); - return; - } - List cols = new ArrayList<>(tableDefinition.getColumnDefinitions().values()); - String sql = generateInsertPrepareSql(offset.getPosition().getSchema(), offset.getPosition().getTableName(), - cols); - DruidPooledConnection connection = null; - PreparedStatement statement = null; try { - connection = DatabaseConnection.sinkDataSource.getConnection(); - statement = - connection.prepareStatement(sql); - for (Map col : data) { - setPrepareParams(statement, col, cols); - log.info("insert sql {}", statement.toString()); - statement.addBatch(); - } - statement.executeBatch(); - connection.commit(); - } catch (SQLException e) { - log.warn("full sink process schema [{}] table [{}] connector write fail", tableDefinition.getSchemaName(), tableDefinition.getTableName(), - e); - LockSupport.parkNanos(3000 * 1000L); - } catch (Exception e) { - log.error("full sink process schema [{}] table [{}] catch unknown exception", tableDefinition.getSchemaName(), - tableDefinition.getTableName(), e); - try { - if (connection != null && !connection.isClosed()) { - connection.rollback(); - } - } catch (SQLException rollback) { - log.warn("full sink process schema [{}] table [{}] rollback fail", tableDefinition.getSchemaName(), - tableDefinition.getTableName(), e); - } - } finally { - if (statement != null) { - try { - statement.close(); - } catch (SQLException e) { - log.info("close prepare statement fail", e); - } - } - - if (connection != null) { - try { - connection.close(); - } catch (SQLException e) { - log.info("close db connection fail", e); - } - } - } - } - - private void setPrepareParams(PreparedStatement preparedStatement, Map col, List columnDefs) throws Exception { - for (int i = 0; i < columnDefs.size(); i++) { - writeColumn(preparedStatement, i + 1, columnDefs.get(i), col.get(columnDefs.get(i).getName())); + queue.put(sinkRecords); + } catch (InterruptedException e) { + throw new RuntimeException(e); } } - public void writeColumn(PreparedStatement ps, int index, MySQLColumnDef colType, Object value) throws Exception { - if (colType == null) { - String colVal = null; - if (value != null) { - colVal = value.toString(); - } - if (colVal == null) { - ps.setNull(index, Types.VARCHAR); - } else { - ps.setString(index, colVal); - } - } else if (value == null) { - ps.setNull(index, colType.getJdbcType().getVendorTypeNumber()); - } else { - switch (colType.getType()) { - case TINYINT: - case SMALLINT: - case MEDIUMINT: - case INT: - Long longValue = SqlUtils.toLong(value); - if (longValue == null) { - ps.setNull(index, 4); - return; - } else { - ps.setLong(index, longValue); - return; - } - case BIGINT: - case DECIMAL: - BigDecimal bigDecimalValue = SqlUtils.toBigDecimal(value); - if (bigDecimalValue == null) { - ps.setNull(index, 3); - return; - } else { - ps.setBigDecimal(index, bigDecimalValue); - return; - } - case FLOAT: - case DOUBLE: - Double doubleValue = SqlUtils.toDouble(value); - if (doubleValue == null) { - ps.setNull(index, 8); - } else { - ps.setDouble(index, doubleValue); - } - return; - case DATE: - case DATETIME: - case TIMESTAMP: - LocalDateTime dateValue = null; - if (!SqlUtils.isZeroTime(value)) { - try { - dateValue = SqlUtils.toLocalDateTime(value); - } catch (Exception e) { - ps.setString(index, SqlUtils.convertToString(value)); - return; - } - } else if (StringUtils.isNotBlank(config.getZeroDate())) { - dateValue = SqlUtils.toLocalDateTime(config.getZeroDate()); - } else { - ps.setObject(index, value); - return; - } - if (dateValue == null) { - ps.setNull(index, Types.TIMESTAMP); - } else { - ps.setString(index, dataTimePattern.format(dateValue)); - } - return; - case TIME: - String timeValue = SqlUtils.toMySqlTime(value); - if (StringUtils.isBlank(timeValue)) { - ps.setNull(index, 12); - return; - } else { - ps.setString(index, timeValue); - return; - } - case YEAR: - LocalDateTime yearValue = null; - if (!SqlUtils.isZeroTime(value)) { - yearValue = SqlUtils.toLocalDateTime(value); - } else if (StringUtils.isNotBlank(config.getZeroDate())) { - yearValue = SqlUtils.toLocalDateTime(config.getZeroDate()); - } else { - ps.setInt(index, 0); - return; - } - if (yearValue == null) { - ps.setNull(index, 4); - } else { - ps.setInt(index, yearValue.getYear()); - } - return; - case CHAR: - case VARCHAR: - case TINYTEXT: - case TEXT: - case MEDIUMTEXT: - case LONGTEXT: - case ENUM: - case SET: - String strValue = value.toString(); - if (strValue == null) { - ps.setNull(index, Types.VARCHAR); - return; - } else { - ps.setString(index, strValue); - return; - } - case JSON: - String jsonValue = value.toString(); - if (jsonValue == null) { - ps.setNull(index, Types.VARCHAR); - } else { - ps.setString(index, jsonValue); - } - return; - case BIT: - if (value instanceof Boolean) { - byte[] arrayBoolean = new byte[1]; - arrayBoolean[0] = (byte) (Boolean.TRUE.equals(value) ? 1 : 0); - ps.setBytes(index, arrayBoolean); - return; - } else if (value instanceof Number) { - ps.setBytes(index, SqlUtils.numberToBinaryArray((Number) value)); - return; - } else if ((value instanceof byte[]) || value.toString().startsWith("0x") || value.toString().startsWith("0X")) { - byte[] arrayBoolean = SqlUtils.toBytes(value); - if (arrayBoolean == null || arrayBoolean.length == 0) { - ps.setNull(index, Types.BIT); - return; - } else { - ps.setBytes(index, arrayBoolean); - return; - } - } else { - ps.setBytes(index, SqlUtils.numberToBinaryArray(SqlUtils.toInt(value))); - return; - } - case BINARY: - case VARBINARY: - case TINYBLOB: - case BLOB: - case MEDIUMBLOB: - case LONGBLOB: - byte[] binaryValue = SqlUtils.toBytes(value); - if (binaryValue == null) { - ps.setNull(index, Types.BINARY); - return; - } else { - ps.setBytes(index, binaryValue); - return; - } - case GEOMETRY: - case GEOMETRY_COLLECTION: - case GEOM_COLLECTION: - case POINT: - case LINESTRING: - case POLYGON: - case MULTIPOINT: - case MULTILINESTRING: - case MULTIPOLYGON: - String geoValue = SqlUtils.toGeometry(value); - if (geoValue == null) { - ps.setNull(index, Types.VARCHAR); - return; - } - ps.setString(index, geoValue); - return; - default: - throw new UnsupportedOperationException("columnType '" + colType + "' Unsupported."); - } - } - } - - private String generateInsertPrepareSql(String schema, String table, List cols) { - StringBuilder builder = new StringBuilder(); - builder.append("INSERT IGNORE INTO "); - builder.append(Constants.MySQLQuot); - builder.append(schema); - builder.append(Constants.MySQLQuot); - builder.append("."); - builder.append(Constants.MySQLQuot); - builder.append(table); - builder.append(Constants.MySQLQuot); - StringBuilder columns = new StringBuilder(); - StringBuilder values = new StringBuilder(); - for (MySQLColumnDef colInfo : cols) { - if (columns.length() > 0) { - columns.append(", "); - values.append(", "); - } - String wrapName = Constants.MySQLQuot + colInfo.getName() + Constants.MySQLQuot; - columns.append(wrapName); - values.append(colInfo.getType() == null ? "?" : colInfo.getType().genPrepareStatement4Insert()); - } - builder.append("(").append(columns).append(")"); - builder.append(" VALUES "); - builder.append("(").append(values).append(")"); - return builder.toString(); - } - - } 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 4137123922..cb50dc5648 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 @@ -17,42 +17,27 @@ package org.apache.eventmesh.connector.canal.sink.connector; +import org.apache.eventmesh.common.EventMeshThreadFactory; 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.CanalSinkFullConfig; -import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.Constants; -import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.MySQLColumnDef; -import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.MySQLTableDef; import org.apache.eventmesh.common.exception.EventMeshException; -import org.apache.eventmesh.common.remote.offset.canal.CanalFullRecordOffset; -import org.apache.eventmesh.common.utils.JsonUtils; import org.apache.eventmesh.connector.canal.DatabaseConnection; -import org.apache.eventmesh.connector.canal.SqlUtils; 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; 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.eventmesh.openconnect.util.ConfigUtil; -import org.apache.commons.lang3.StringUtils; - -import java.math.BigDecimal; -import java.sql.PreparedStatement; -import java.sql.SQLException; -import java.sql.Types; -import java.time.LocalDateTime; -import java.time.format.DateTimeFormatter; -import java.util.ArrayList; +import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.concurrent.locks.LockSupport; - -import com.alibaba.druid.pool.DruidPooledConnection; -import com.fasterxml.jackson.core.type.TypeReference; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import lombok.extern.slf4j.Slf4j; @@ -61,7 +46,9 @@ public class CanalSinkFullConnector implements Sink, ConnectorCreateService> queue = new LinkedBlockingQueue<>(10000); + private final AtomicBoolean flag = new AtomicBoolean(true); @Override public void start() throws Exception { @@ -70,7 +57,23 @@ public void start() throws Exception { @Override public void stop() throws Exception { - + flag.set(false); + if (!executor.isShutdown()) { + executor.shutdown(); + try { + if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { + log.warn("wait thread pool shutdown timeout, it will shutdown now"); + executor.shutdownNow(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + log.info("shutdown thread pool fail"); + } + } + if (DatabaseConnection.sinkDataSource != null) { + DatabaseConnection.sinkDataSource.close(); + log.info("data source has been closed"); + } } @Override @@ -106,6 +109,14 @@ private void init() { DatabaseConnection.sinkDataSource.setDefaultAutoCommit(false); tableMgr = new RdbTableMgr(this.config.getSinkConnectorConfig(), DatabaseConnection.sinkDataSource); + executor = new ThreadPoolExecutor(config.getParallel(), config.getParallel(), 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(), new EventMeshThreadFactory("canal-sink-full")); + List consumers = new LinkedList<>(); + for (int i = 0; i < config.getParallel(); i++) { + CanalFullConsumer canalFullConsumer = new CanalFullConsumer(queue, tableMgr, config); + consumers.add(canalFullConsumer); + } + consumers.forEach(c -> executor.execute(() -> c.start(flag))); } @Override @@ -131,309 +142,12 @@ public void put(List sinkRecords) { } return; } - ConnectRecord record = sinkRecords.get(0); - List> data = - JsonUtils.parseTypeReferenceObject((byte[]) record.getData(), new TypeReference>>() { - }); - if (data == null || data.isEmpty()) { - if (log.isDebugEnabled()) { - log.debug("[{}] got rows data is none", this.getClass()); - } - return; - } - CanalFullRecordOffset offset = (CanalFullRecordOffset) record.getPosition().getRecordOffset(); - if (offset == null || offset.getPosition() == null) { - if (log.isDebugEnabled()) { - log.debug("[{}] got canal full offset is none", this.getClass()); - } - return; - } - - MySQLTableDef tableDefinition = (MySQLTableDef) tableMgr.getTable(offset.getPosition().getSchema(), offset.getPosition().getTableName()); - if (tableDefinition == null) { - log.warn("target schema [{}] table [{}] is not exists", offset.getPosition().getSchema(), offset.getPosition().getTableName()); - return; - } - List cols = new ArrayList<>(tableDefinition.getColumnDefinitions().values()); - String sql = generateInsertPrepareSql(offset.getPosition().getSchema(), offset.getPosition().getTableName(), - cols); - DruidPooledConnection connection = null; - PreparedStatement statement = null; try { - connection = DatabaseConnection.sinkDataSource.getConnection(); - statement = - connection.prepareStatement(sql); - for (Map col : data) { - setPrepareParams(statement, col, cols); - log.info("insert sql {}", statement.toString()); - statement.addBatch(); - } - statement.executeBatch(); - connection.commit(); - record.getCallback().onSuccess(convertToSendResult(record)); - } catch (SQLException e) { - log.warn("full sink process schema [{}] table [{}] connector write fail", tableDefinition.getSchemaName(), tableDefinition.getTableName(), - e); - LockSupport.parkNanos(3000 * 1000L); - record.getCallback().onException(buildSendExceptionContext(record, e)); - } catch (Exception e) { - log.error("full sink process schema [{}] table [{}] catch unknown exception", tableDefinition.getSchemaName(), - tableDefinition.getTableName(), e); - record.getCallback().onException(buildSendExceptionContext(record, e)); - try { - if (connection != null && !connection.isClosed()) { - connection.rollback(); - } - } catch (SQLException rollback) { - log.warn("full sink process schema [{}] table [{}] rollback fail", tableDefinition.getSchemaName(), - tableDefinition.getTableName(), e); - } - } finally { - if (statement != null) { - try { - statement.close(); - } catch (SQLException e) { - log.info("close prepare statement fail", e); - } - } - - if (connection != null) { - try { - connection.close(); - } catch (SQLException e) { - log.info("close db connection fail", e); - } - } - } - } - - 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 setPrepareParams(PreparedStatement preparedStatement, Map col, List columnDefs) throws Exception { - for (int i = 0; i < columnDefs.size(); i++) { - writeColumn(preparedStatement, i + 1, columnDefs.get(i), col.get(columnDefs.get(i).getName())); - } - } - - public void writeColumn(PreparedStatement ps, int index, MySQLColumnDef colType, Object value) throws Exception { - if (colType == null) { - String colVal = null; - if (value != null) { - colVal = value.toString(); - } - if (colVal == null) { - ps.setNull(index, Types.VARCHAR); - } else { - ps.setString(index, colVal); - } - } else if (value == null) { - ps.setNull(index, colType.getJdbcType().getVendorTypeNumber()); - } else { - switch (colType.getType()) { - case TINYINT: - case SMALLINT: - case MEDIUMINT: - case INT: - Long longValue = SqlUtils.toLong(value); - if (longValue == null) { - ps.setNull(index, 4); - return; - } else { - ps.setLong(index, longValue); - return; - } - case BIGINT: - case DECIMAL: - BigDecimal bigDecimalValue = SqlUtils.toBigDecimal(value); - if (bigDecimalValue == null) { - ps.setNull(index, 3); - return; - } else { - ps.setBigDecimal(index, bigDecimalValue); - return; - } - case FLOAT: - case DOUBLE: - Double doubleValue = SqlUtils.toDouble(value); - if (doubleValue == null) { - ps.setNull(index, 8); - } else { - ps.setDouble(index, doubleValue); - } - return; - case DATE: - case DATETIME: - case TIMESTAMP: - LocalDateTime dateValue = null; - if (!SqlUtils.isZeroTime(value)) { - try { - dateValue = SqlUtils.toLocalDateTime(value); - } catch (Exception e) { - ps.setString(index, SqlUtils.convertToString(value)); - return; - } - } else if (StringUtils.isNotBlank(config.getZeroDate())) { - dateValue = SqlUtils.toLocalDateTime(config.getZeroDate()); - } else { - ps.setObject(index, value); - return; - } - if (dateValue == null) { - ps.setNull(index, Types.TIMESTAMP); - } else { - ps.setString(index, dataTimePattern.format(dateValue)); - } - return; - case TIME: - String timeValue = SqlUtils.toMySqlTime(value); - if (StringUtils.isBlank(timeValue)) { - ps.setNull(index, 12); - return; - } else { - ps.setString(index, timeValue); - return; - } - case YEAR: - LocalDateTime yearValue = null; - if (!SqlUtils.isZeroTime(value)) { - yearValue = SqlUtils.toLocalDateTime(value); - } else if (StringUtils.isNotBlank(config.getZeroDate())) { - yearValue = SqlUtils.toLocalDateTime(config.getZeroDate()); - } else { - ps.setInt(index, 0); - return; - } - if (yearValue == null) { - ps.setNull(index, 4); - } else { - ps.setInt(index, yearValue.getYear()); - } - return; - case CHAR: - case VARCHAR: - case TINYTEXT: - case TEXT: - case MEDIUMTEXT: - case LONGTEXT: - case ENUM: - case SET: - String strValue = value.toString(); - if (strValue == null) { - ps.setNull(index, Types.VARCHAR); - return; - } else { - ps.setString(index, strValue); - return; - } - case JSON: - String jsonValue = value.toString(); - if (jsonValue == null) { - ps.setNull(index, Types.VARCHAR); - } else { - ps.setString(index, jsonValue); - } - return; - case BIT: - if (value instanceof Boolean) { - byte[] arrayBoolean = new byte[1]; - arrayBoolean[0] = (byte) (Boolean.TRUE.equals(value) ? 1 : 0); - ps.setBytes(index, arrayBoolean); - return; - } else if (value instanceof Number) { - ps.setBytes(index, SqlUtils.numberToBinaryArray((Number) value)); - return; - } else if ((value instanceof byte[]) || value.toString().startsWith("0x") || value.toString().startsWith("0X")) { - byte[] arrayBoolean = SqlUtils.toBytes(value); - if (arrayBoolean == null || arrayBoolean.length == 0) { - ps.setNull(index, Types.BIT); - return; - } else { - ps.setBytes(index, arrayBoolean); - return; - } - } else { - ps.setBytes(index, SqlUtils.numberToBinaryArray(SqlUtils.toInt(value))); - return; - } - case BINARY: - case VARBINARY: - case TINYBLOB: - case BLOB: - case MEDIUMBLOB: - case LONGBLOB: - byte[] binaryValue = SqlUtils.toBytes(value); - if (binaryValue == null) { - ps.setNull(index, Types.BINARY); - return; - } else { - ps.setBytes(index, binaryValue); - return; - } - case GEOMETRY: - case GEOMETRY_COLLECTION: - case GEOM_COLLECTION: - case POINT: - case LINESTRING: - case POLYGON: - case MULTIPOINT: - case MULTILINESTRING: - case MULTIPOLYGON: - String geoValue = SqlUtils.toGeometry(value); - if (geoValue == null) { - ps.setNull(index, Types.VARCHAR); - return; - } - ps.setString(index, geoValue); - return; - default: - throw new UnsupportedOperationException("columnType '" + colType + "' Unsupported."); - } + queue.put(sinkRecords); + } catch (InterruptedException e) { + throw new RuntimeException(e); } - } - private String generateInsertPrepareSql(String schema, String table, List cols) { - StringBuilder builder = new StringBuilder(); - builder.append("INSERT IGNORE INTO "); - builder.append(Constants.MySQLQuot); - builder.append(schema); - builder.append(Constants.MySQLQuot); - builder.append("."); - builder.append(Constants.MySQLQuot); - builder.append(table); - builder.append(Constants.MySQLQuot); - StringBuilder columns = new StringBuilder(); - StringBuilder values = new StringBuilder(); - for (MySQLColumnDef colInfo : cols) { - if (columns.length() > 0) { - columns.append(", "); - values.append(", "); - } - String wrapName = Constants.MySQLQuot + colInfo.getName() + Constants.MySQLQuot; - columns.append(wrapName); - values.append(colInfo.getType() == null ? "?" : colInfo.getType().genPrepareStatement4Insert()); - } - builder.append("(").append(columns).append(")"); - builder.append(" VALUES "); - builder.append("(").append(values).append(")"); - return builder.toString(); } - } diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkIncrementConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkIncrementConnector.java index e165a5ffe6..84373ae7a7 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkIncrementConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/sink/connector/CanalSinkIncrementConnector.java @@ -680,7 +680,7 @@ public int getBatchSize() { } catch (Exception e) { // rollback status.setRollbackOnly(); - throw new RuntimeException("Failed to execute batch with GTID", e); + throw new RuntimeException("Failed to execute batch ", e); } finally { lobCreator.close(); } 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 5a6ceb7c3f..d7388c628b 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 @@ -69,6 +69,7 @@ public static Map> parse(CanalSourceIncrementConf // if not gtid mode, need check weather the entry is loopback by specified column value needSync = checkNeedSync(sourceConfig, rowChange); if (needSync) { + log.debug("entry evenType {}|rowChange {}", rowChange.getEventType(), rowChange); transactionDataBuffer.add(entry); } } @@ -76,14 +77,27 @@ public static Map> parse(CanalSourceIncrementConf case TRANSACTIONEND: parseRecordListWithEntryBuffer(sourceConfig, recordList, transactionDataBuffer, tables); if (!recordList.isEmpty()) { - recordMap.put(entry.getHeader().getLogfileOffset(), recordList); + List transactionEndList = new ArrayList<>(recordList); + recordMap.put(entry.getHeader().getLogfileOffset(), transactionEndList); } + recordList.clear(); transactionDataBuffer.clear(); break; default: break; } } + + // add last data in transactionDataBuffer, in case no TRANSACTIONEND + parseRecordListWithEntryBuffer(sourceConfig, recordList, transactionDataBuffer, tables); + if (!recordList.isEmpty()) { + List transactionEndList = new ArrayList<>(recordList); + CanalConnectRecord lastCanalConnectRecord = transactionEndList.get(transactionEndList.size() - 1); + recordMap.put(lastCanalConnectRecord.getBinLogOffset(), transactionEndList); + } + recordList.clear(); + transactionDataBuffer.clear(); + } catch (Exception e) { throw new RuntimeException(e); } @@ -118,6 +132,9 @@ private static void parseRecordListWithEntryBuffer(CanalSourceIncrementConfig so private static boolean checkNeedSync(CanalSourceIncrementConfig sourceConfig, RowChange rowChange) { Column markedColumn = null; CanalEntry.EventType eventType = rowChange.getEventType(); + if (StringUtils.isEmpty(sourceConfig.getNeedSyncMarkTableColumnName())) { + return true; + } if (eventType.equals(CanalEntry.EventType.DELETE)) { markedColumn = getColumnIgnoreCase(rowChange.getRowDatas(0).getBeforeColumnsList(), sourceConfig.getNeedSyncMarkTableColumnName()); diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalFullProducer.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalFullProducer.java index c0b2063d28..644b77247d 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalFullProducer.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalFullProducer.java @@ -32,6 +32,7 @@ import org.apache.eventmesh.openconnect.offsetmgmt.api.data.ConnectRecord; import java.math.BigDecimal; +import java.math.RoundingMode; import java.nio.charset.StandardCharsets; import java.sql.Connection; import java.sql.PreparedStatement; @@ -45,17 +46,22 @@ import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.BlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.LockSupport; import javax.sql.DataSource; +import com.google.common.util.concurrent.RateLimiter; + +import lombok.Setter; import lombok.extern.slf4j.Slf4j; @@ -64,26 +70,34 @@ public class CanalFullProducer { private BlockingQueue> queue; private final DataSource dataSource; private final MySQLTableDef tableDefinition; - private final TableFullPosition position; + private final TableFullPosition tableFullPosition; + private final JobRdbFullPosition startPosition; private static final int LIMIT = 2048; private final int flushSize; private final AtomicReference choosePrimaryKey = new AtomicReference<>(null); private static final DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd"); private static final DateTimeFormatter DATE_STAMP_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + private AtomicLong scanCount = new AtomicLong(0); + private final RateLimiter pageLimiter; + @Setter + private RateLimiter recordLimiter; public CanalFullProducer(BlockingQueue> queue, DataSource dataSource, - MySQLTableDef tableDefinition, TableFullPosition position, int flushSize) { + MySQLTableDef tableDefinition, JobRdbFullPosition startPosition, int flushSize, int pagePerSecond) { this.queue = queue; this.dataSource = dataSource; this.tableDefinition = tableDefinition; - this.position = position; + this.startPosition = startPosition; + this.tableFullPosition = JsonUtils.parseObject(startPosition.getPrimaryKeyRecords(), TableFullPosition.class); + this.scanCount.set(startPosition.getHandledRecordCount()); this.flushSize = flushSize; + this.pageLimiter = RateLimiter.create(pagePerSecond); } public void choosePrimaryKey() { for (RdbColumnDefinition col : tableDefinition.getColumnDefinitions().values()) { - if (position.getCurPrimaryKeyCols().get(col.getName()) != null) { + if (tableFullPosition.getCurPrimaryKeyCols().get(col.getName()) != null) { // random choose the first primary key from the table choosePrimaryKey.set(col.getName()); log.info("schema [{}] table [{}] choose primary key [{}]", tableDefinition.getSchemaName(), tableDefinition.getTableName(), @@ -101,8 +115,11 @@ public void start(AtomicBoolean flag) { boolean isFirstSelect = true; List> rows = new LinkedList<>(); while (flag.get()) { + // acquire a permit before each database read + pageLimiter.acquire(); + String scanSql = generateScanSql(isFirstSelect); - log.info("scan sql is [{}] , cur position [{}]", scanSql, JsonUtils.toJSONString(position.getCurPrimaryKeyCols())); + log.info("scan sql is [{}] , cur position [{}]", scanSql, JsonUtils.toJSONString(tableFullPosition.getCurPrimaryKeyCols())); try (Connection connection = dataSource.getConnection(); PreparedStatement statement = connection.prepareStatement(scanSql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)) { @@ -119,12 +136,13 @@ public void start(AtomicBoolean flag) { } lastCol = columnValues; rows.add(lastCol); + this.scanCount.incrementAndGet(); if (rows.size() < flushSize) { continue; } refreshPosition(lastCol); // may be not reach - commitConnectRecord(rows); + commitConnectRecord(rows, false, this.scanCount.get(), startPosition); rows = new LinkedList<>(); } @@ -132,7 +150,7 @@ public void start(AtomicBoolean flag) { log.info("full scan db [{}] table [{}] finish", tableDefinition.getSchemaName(), tableDefinition.getTableName()); // commit the last record if rows.size() < flushSize - commitConnectRecord(rows); + commitConnectRecord(rows, true, this.scanCount.get(), startPosition); return; } refreshPosition(lastCol); @@ -157,26 +175,44 @@ public void start(AtomicBoolean flag) { } } - private void commitConnectRecord(List> rows) throws InterruptedException { + private void commitConnectRecord(List> rows, boolean isFinished, long migratedCount, JobRdbFullPosition position) + throws InterruptedException { if (rows == null || rows.isEmpty()) { return; } JobRdbFullPosition jobRdbFullPosition = new JobRdbFullPosition(); - jobRdbFullPosition.setPrimaryKeyRecords(JsonUtils.toJSONString(position)); + jobRdbFullPosition.setPrimaryKeyRecords(JsonUtils.toJSONString(tableFullPosition)); jobRdbFullPosition.setTableName(tableDefinition.getTableName()); jobRdbFullPosition.setSchema(tableDefinition.getSchemaName()); + jobRdbFullPosition.setFinished(isFinished); + jobRdbFullPosition.setHandledRecordCount(migratedCount); + jobRdbFullPosition.setMaxCount(position.getMaxCount()); + if (isFinished) { + jobRdbFullPosition.setPercent(new BigDecimal("100")); + } else { + double num = 100.0d * ((double) migratedCount) * 1.0d / (double) position.getMaxCount(); + String number = Double.toString(num); + BigDecimal percent = new BigDecimal(number).setScale(2, RoundingMode.HALF_UP); + jobRdbFullPosition.setPercent(percent); + } CanalFullRecordOffset offset = new CanalFullRecordOffset(); offset.setPosition(jobRdbFullPosition); CanalFullRecordPartition partition = new CanalFullRecordPartition(); + Map dataMap = new HashMap<>(); + dataMap.put("data", JsonUtils.toJSONString(rows)); + dataMap.put("partition", JsonUtils.toJSONString(partition)); + dataMap.put("offset", JsonUtils.toJSONString(offset)); ArrayList records = new ArrayList<>(); - byte[] rowsData = JsonUtils.toJSONString(rows).getBytes(StandardCharsets.UTF_8); - records.add(new ConnectRecord(partition, offset, System.currentTimeMillis(), rowsData)); + records.add( + new ConnectRecord(partition, offset, System.currentTimeMillis(), JsonUtils.toJSONString(dataMap).getBytes(StandardCharsets.UTF_8))); + // global limiter, 100 records per second default + recordLimiter.acquire(); queue.put(records); } private boolean checkIsScanFinish(Map lastCol) { Object lastPrimaryValue = lastCol.get(choosePrimaryKey.get()); - Object maxPrimaryValue = position.getMaxPrimaryKeyCols().get(choosePrimaryKey.get()); + Object maxPrimaryValue = tableFullPosition.getMaxPrimaryKeyCols().get(choosePrimaryKey.get()); if (lastPrimaryValue instanceof Number) { BigDecimal last = new BigDecimal(String.valueOf(lastPrimaryValue)); BigDecimal max = @@ -189,22 +225,22 @@ private boolean checkIsScanFinish(Map lastCol) { return false; } - public Object readColumn(ResultSet rs, String col, CanalMySQLType colType) throws Exception { - if (col == null || rs.wasNull()) { - return null; - } + public Object readColumn(ResultSet rs, String colName, CanalMySQLType colType) throws Exception { switch (colType) { case TINYINT: case SMALLINT: case MEDIUMINT: case INT: - Long valueLong = rs.getLong(col); + Long valueLong = rs.getLong(colName); + if (rs.wasNull()) { + return null; + } if (valueLong.compareTo((long) Integer.MAX_VALUE) > 0) { return valueLong; } return valueLong.intValue(); case BIGINT: - String v = rs.getString(col); + String v = rs.getString(colName); if (v == null) { return null; } @@ -216,16 +252,20 @@ public Object readColumn(ResultSet rs, String col, CanalMySQLType colType) throw case FLOAT: case DOUBLE: case DECIMAL: - return rs.getBigDecimal(col); + return rs.getBigDecimal(colName); case DATE: - return rs.getObject(col, LocalDate.class); + return rs.getObject(colName, LocalDate.class); case TIME: - return rs.getObject(col, LocalTime.class); + return rs.getObject(colName, LocalTime.class); case DATETIME: case TIMESTAMP: - return rs.getObject(col, LocalDateTime.class); + return rs.getObject(colName, LocalDateTime.class); case YEAR: - return rs.getInt(col); + int year = rs.getInt(colName); + if (rs.wasNull()) { + return null; + } + return year; case CHAR: case VARCHAR: case TINYTEXT: @@ -235,7 +275,7 @@ public Object readColumn(ResultSet rs, String col, CanalMySQLType colType) throw case ENUM: case SET: case JSON: - return rs.getString(col); + return rs.getString(colName); case BIT: case BINARY: case VARBINARY: @@ -243,7 +283,7 @@ public Object readColumn(ResultSet rs, String col, CanalMySQLType colType) throw case BLOB: case MEDIUMBLOB: case LONGBLOB: - return rs.getBytes(col); + return rs.getBytes(colName); case GEOMETRY: case GEOMETRY_COLLECTION: case GEOM_COLLECTION: @@ -253,23 +293,23 @@ public Object readColumn(ResultSet rs, String col, CanalMySQLType colType) throw case MULTIPOINT: case MULTILINESTRING: case MULTIPOLYGON: - byte[] geo = rs.getBytes(col); + byte[] geo = rs.getBytes(colName); if (geo == null) { return null; } return SqlUtils.toGeometry(geo); default: - return rs.getObject(col); + return rs.getObject(colName); } } private void refreshPosition(Map lastCol) { Map nextPosition = new LinkedHashMap<>(); - for (Map.Entry entry : position.getCurPrimaryKeyCols().entrySet()) { + for (Map.Entry entry : tableFullPosition.getCurPrimaryKeyCols().entrySet()) { nextPosition.put(entry.getKey(), lastCol.get(entry.getKey())); } - position.setCurPrimaryKeyCols(nextPosition); + tableFullPosition.setCurPrimaryKeyCols(nextPosition); } private void setPrepareStatementValue(PreparedStatement statement) throws SQLException { @@ -278,7 +318,7 @@ private void setPrepareStatementValue(PreparedStatement statement) throws SQLExc return; } RdbColumnDefinition columnDefinition = tableDefinition.getColumnDefinitions().get(colName); - Object value = position.getCurPrimaryKeyCols().get(colName); + Object value = tableFullPosition.getCurPrimaryKeyCols().get(colName); String str; switch (columnDefinition.getJdbcType()) { case BIT: diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceCheckConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceCheckConnector.java index bd85f03240..4d3e569dcd 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceCheckConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceCheckConnector.java @@ -20,16 +20,15 @@ import org.apache.eventmesh.common.AbstractComponent; import org.apache.eventmesh.common.EventMeshThreadFactory; import org.apache.eventmesh.common.config.connector.Config; -import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSourceFullConfig; +import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSourceCheckConfig; +import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSourceConfig; import org.apache.eventmesh.common.config.connector.rdb.canal.JobRdbFullPosition; import org.apache.eventmesh.common.config.connector.rdb.canal.RdbDBDefinition; import org.apache.eventmesh.common.config.connector.rdb.canal.RdbTableDefinition; import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.MySQLTableDef; import org.apache.eventmesh.common.exception.EventMeshException; -import org.apache.eventmesh.common.utils.JsonUtils; import org.apache.eventmesh.connector.canal.DatabaseConnection; -import org.apache.eventmesh.connector.canal.source.position.CanalFullPositionMgr; -import org.apache.eventmesh.connector.canal.source.position.TableFullPosition; +import org.apache.eventmesh.connector.canal.source.position.CanalCheckPositionMgr; import org.apache.eventmesh.connector.canal.source.table.RdbSimpleTable; import org.apache.eventmesh.connector.canal.source.table.RdbTableMgr; import org.apache.eventmesh.openconnect.api.ConnectorCreateService; @@ -37,67 +36,86 @@ 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.util.ConfigUtil; import java.util.LinkedList; import java.util.List; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.util.concurrent.RateLimiter; + import lombok.extern.slf4j.Slf4j; @Slf4j public class CanalSourceCheckConnector extends AbstractComponent implements Source, ConnectorCreateService { - private CanalSourceFullConfig config; - private CanalFullPositionMgr positionMgr; + private CanalSourceCheckConfig config; + private CanalCheckPositionMgr positionMgr; private RdbTableMgr tableMgr; private ThreadPoolExecutor executor; - private BlockingQueue> queue; + private final ScheduledExecutorService scheduledThreadPoolExecutor = Executors.newSingleThreadScheduledExecutor(); + private final BlockingQueue> queue = new LinkedBlockingQueue<>(10000); private final AtomicBoolean flag = new AtomicBoolean(true); - private long maxPollWaitTime; + private RateLimiter globalLimiter; @Override protected void run() throws Exception { - this.tableMgr.start(); - this.positionMgr.start(); - if (positionMgr.isFinished()) { - log.info("connector [{}] has finished the job", config.getSourceConnectorConfig().getConnectorName()); - return; - } - executor = new ThreadPoolExecutor(config.getParallel(), config.getParallel(), 0L, TimeUnit.MILLISECONDS, - new LinkedBlockingQueue<>(), new EventMeshThreadFactory("canal-source-full")); - List producers = new LinkedList<>(); - if (config.getSourceConnectorConfig().getDatabases() != null) { - for (RdbDBDefinition db : config.getSourceConnectorConfig().getDatabases()) { - for (RdbTableDefinition table : db.getTables()) { - try { - log.info("it will create producer of db [{}] table [{}]", db.getSchemaName(), table.getTableName()); - RdbSimpleTable simpleTable = new RdbSimpleTable(db.getSchemaName(), table.getTableName()); - JobRdbFullPosition position = positionMgr.getPosition(simpleTable); - if (position == null) { - throw new EventMeshException(String.format("db [%s] table [%s] have none position info", - db.getSchemaName(), table.getTableName())); - } - RdbTableDefinition tableDefinition = tableMgr.getTable(simpleTable); - if (tableDefinition == null) { - throw new EventMeshException(String.format("db [%s] table [%s] have none table definition info", - db.getSchemaName(), table.getTableName())); + scheduledThreadPoolExecutor.scheduleAtFixedRate(() -> { + try { + this.tableMgr.start(); + } catch (Exception e) { + log.error("start tableMgr fail", e); + throw new RuntimeException(e); + } + try { + this.positionMgr.start(); + } catch (Exception e) { + throw new RuntimeException(e); + } + // if (positionMgr.isFinished()) { + // log.info("connector [{}] has finished the job", config.getSourceConnectorConfig().getConnectorName()); + // return; + // } + executor = new ThreadPoolExecutor(config.getParallel(), config.getParallel(), 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(), new EventMeshThreadFactory("canal-source-check")); + List producers = new LinkedList<>(); + if (config.getSourceConnectorConfig().getDatabases() != null) { + for (RdbDBDefinition db : config.getSourceConnectorConfig().getDatabases()) { + for (RdbTableDefinition table : db.getTables()) { + try { + log.info("it will create producer of db [{}] table [{}]", db.getSchemaName(), table.getTableName()); + RdbSimpleTable simpleTable = new RdbSimpleTable(db.getSchemaName(), table.getTableName()); + JobRdbFullPosition position = positionMgr.getPosition(simpleTable); + if (position == null) { + throw new EventMeshException(String.format("db [%s] table [%s] have none position info", + db.getSchemaName(), table.getTableName())); + } + RdbTableDefinition tableDefinition = tableMgr.getTable(simpleTable); + if (tableDefinition == null) { + throw new EventMeshException(String.format("db [%s] table [%s] have none table definition info", + db.getSchemaName(), table.getTableName())); + } + CanalFullProducer producer = + new CanalFullProducer(queue, DatabaseConnection.sourceDataSource, (MySQLTableDef) tableDefinition, + position, config.getFlushSize(), config.getPagePerSecond()); + producer.setRecordLimiter(globalLimiter); + producers.add(producer); + } catch (Exception e) { + log.error("create schema [{}] table [{}] producers fail", db.getSchemaName(), + table.getTableName(), e); } - - producers.add(new CanalFullProducer(queue, DatabaseConnection.sourceDataSource, (MySQLTableDef) tableDefinition, - JsonUtils.parseObject(position.getPrimaryKeyRecords(), TableFullPosition.class), - config.getFlushSize())); - } catch (Exception e) { - log.error("create schema [{}] table [{}] producers fail", db.getSchemaName(), - table.getTableName(), e); } } } - } - producers.forEach(p -> executor.execute(() -> p.start(flag))); + producers.forEach(p -> executor.execute(() -> p.start(flag))); + + }, 0, config.getExecutePeriod(), TimeUnit.SECONDS); } @Override @@ -115,6 +133,18 @@ protected void shutdown() throws Exception { log.info("shutdown thread pool fail"); } } + if (!scheduledThreadPoolExecutor.isShutdown()) { + scheduledThreadPoolExecutor.shutdown(); + try { + if (!scheduledThreadPoolExecutor.awaitTermination(5, TimeUnit.SECONDS)) { + log.warn("wait scheduledThreadPoolExecutor shutdown timeout, it will shutdown now"); + scheduledThreadPoolExecutor.shutdownNow(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + log.info("shutdown scheduledThreadPoolExecutor fail"); + } + } if (DatabaseConnection.sourceDataSource != null) { DatabaseConnection.sourceDataSource.close(); log.info("data source has been closed"); @@ -128,12 +158,12 @@ public Source create() { @Override public Class configClass() { - return CanalSourceFullConfig.class; + return CanalSourceCheckConfig.class; } @Override public void init(Config config) throws Exception { - this.config = (CanalSourceFullConfig) config; + this.config = (CanalSourceCheckConfig) config; init(); } @@ -141,15 +171,15 @@ private void init() { DatabaseConnection.sourceConfig = this.config.getSourceConnectorConfig(); DatabaseConnection.initSourceConnection(); this.tableMgr = new RdbTableMgr(config.getSourceConnectorConfig(), DatabaseConnection.sourceDataSource); - this.positionMgr = new CanalFullPositionMgr(config, tableMgr); - this.maxPollWaitTime = config.getPollConfig().getMaxWaitTime(); - this.queue = new LinkedBlockingQueue<>(config.getPollConfig().getCapacity()); + this.positionMgr = new CanalCheckPositionMgr(config, tableMgr); + this.globalLimiter = RateLimiter.create(config.getRecordPerSecond()); } @Override public void init(ConnectorContext connectorContext) throws Exception { SourceConnectorContext sourceConnectorContext = (SourceConnectorContext) connectorContext; - this.config = (CanalSourceFullConfig) sourceConnectorContext.getSourceConfig(); + CanalSourceConfig canalSourceConfig = (CanalSourceConfig) sourceConnectorContext.getSourceConfig(); + this.config = ConfigUtil.parse(canalSourceConfig.getSourceConfig(), CanalSourceCheckConfig.class); init(); } @@ -172,7 +202,7 @@ public void onException(ConnectRecord record) { public List poll() { while (flag.get()) { try { - List records = queue.poll(maxPollWaitTime, TimeUnit.MILLISECONDS); + List records = queue.poll(5, TimeUnit.SECONDS); if (records == null || records.isEmpty()) { continue; } 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 09e2e0dcf7..df28342c39 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 @@ -27,10 +27,8 @@ import org.apache.eventmesh.common.config.connector.rdb.canal.RdbTableDefinition; import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.MySQLTableDef; import org.apache.eventmesh.common.exception.EventMeshException; -import org.apache.eventmesh.common.utils.JsonUtils; import org.apache.eventmesh.connector.canal.DatabaseConnection; import org.apache.eventmesh.connector.canal.source.position.CanalFullPositionMgr; -import org.apache.eventmesh.connector.canal.source.position.TableFullPosition; import org.apache.eventmesh.connector.canal.source.table.RdbSimpleTable; import org.apache.eventmesh.connector.canal.source.table.RdbTableMgr; import org.apache.eventmesh.openconnect.api.connector.ConnectorContext; @@ -47,6 +45,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.util.concurrent.RateLimiter; + import lombok.extern.slf4j.Slf4j; @Slf4j @@ -56,9 +56,9 @@ public class CanalSourceFullConnector extends AbstractComponent implements Sourc private CanalFullPositionMgr positionMgr; private RdbTableMgr tableMgr; private ThreadPoolExecutor executor; - private BlockingQueue> queue; + private final BlockingQueue> queue = new LinkedBlockingQueue<>(10000); private final AtomicBoolean flag = new AtomicBoolean(true); - private long maxPollWaitTime; + private RateLimiter globalLimiter; @Override protected void run() throws Exception { @@ -87,10 +87,11 @@ protected void run() throws Exception { throw new EventMeshException(String.format("db [%s] table [%s] have none table definition info", db.getSchemaName(), table.getTableName())); } - - producers.add(new CanalFullProducer(queue, DatabaseConnection.sourceDataSource, (MySQLTableDef) tableDefinition, - JsonUtils.parseObject(position.getPrimaryKeyRecords(), TableFullPosition.class), - config.getFlushSize())); + CanalFullProducer producer = + new CanalFullProducer(queue, DatabaseConnection.sourceDataSource, (MySQLTableDef) tableDefinition, + position, config.getFlushSize(), config.getPagePerSecond()); + producer.setRecordLimiter(globalLimiter); + producers.add(producer); } catch (Exception e) { log.error("create schema [{}] table [{}] producers fail", db.getSchemaName(), table.getTableName(), e); @@ -138,8 +139,7 @@ private void init() { DatabaseConnection.initSourceConnection(); this.tableMgr = new RdbTableMgr(config.getSourceConnectorConfig(), DatabaseConnection.sourceDataSource); this.positionMgr = new CanalFullPositionMgr(config, tableMgr); - this.maxPollWaitTime = config.getPollConfig().getMaxWaitTime(); - this.queue = new LinkedBlockingQueue<>(config.getPollConfig().getCapacity()); + this.globalLimiter = RateLimiter.create(config.getRecordPerSecond()); } @Override @@ -169,7 +169,7 @@ public void onException(ConnectRecord record) { public List poll() { while (flag.get()) { try { - List records = queue.poll(maxPollWaitTime, TimeUnit.MILLISECONDS); + List records = queue.poll(2, TimeUnit.SECONDS); if (records == null || records.isEmpty()) { continue; } diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceIncrementConnector.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceIncrementConnector.java index 4f7041b478..c6e7603805 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceIncrementConnector.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/connector/CanalSourceIncrementConnector.java @@ -20,6 +20,9 @@ import org.apache.eventmesh.common.config.connector.Config; import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSourceConfig; import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSourceIncrementConfig; +import org.apache.eventmesh.common.config.connector.rdb.canal.RdbDBDefinition; +import org.apache.eventmesh.common.config.connector.rdb.canal.RdbTableDefinition; +import org.apache.eventmesh.common.remote.datasource.DataSourceType; import org.apache.eventmesh.common.remote.offset.RecordPosition; import org.apache.eventmesh.common.remote.offset.canal.CanalRecordOffset; import org.apache.eventmesh.common.remote.offset.canal.CanalRecordPartition; @@ -38,6 +41,8 @@ import java.net.InetSocketAddress; import java.nio.charset.StandardCharsets; +import java.sql.PreparedStatement; +import java.sql.ResultSet; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -47,6 +52,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.LockSupport; +import com.alibaba.druid.pool.DruidDataSource; import com.alibaba.otter.canal.instance.core.CanalInstance; import com.alibaba.otter.canal.instance.core.CanalInstanceGenerator; import com.alibaba.otter.canal.instance.manager.CanalInstanceWithManager; @@ -89,6 +95,12 @@ public class CanalSourceIncrementConnector implements Source { private RdbTableMgr tableMgr; + private static final String SQL_SELECT_RDB_VERSION = "select version() as rdb_version"; + + private static final String SQL_SELECT_SERVER_UUID_IN_MARIADB = "SELECT @@global.server_id as server_uuid"; + + private static final String SQL_SHOW_SERVER_UUID_IN_MYSQL = "SELECT @@server_uuid as server_uuid"; + @Override public Class configClass() { return CanalSourceConfig.class; @@ -108,13 +120,24 @@ public void init(ConnectorContext connectorContext) throws Exception { if (sourceConnectorContext.getRecordPositionList() != null) { this.sourceConfig.setRecordPositions(sourceConnectorContext.getRecordPositionList()); } + // filter: your_database\\.your_table; .*\\..* (all database & table) + tableFilter = buildTableFilters(sourceConfig); - if (StringUtils.isNotEmpty(sourceConfig.getTableFilter())) { - tableFilter = sourceConfig.getTableFilter(); - } if (StringUtils.isNotEmpty(sourceConfig.getFieldFilter())) { fieldFilter = sourceConfig.getFieldFilter(); } + DatabaseConnection.sourceConfig = sourceConfig.getSourceConnectorConfig(); + DatabaseConnection.initSourceConnection(); + + DataSourceType dataSourceType = checkRDBDataSourceType(DatabaseConnection.sourceDataSource); + String serverUUID = queryServerUUID(DatabaseConnection.sourceDataSource, dataSourceType); + if (StringUtils.isNotEmpty(serverUUID)) { + log.info("init source increment connector, serverUUID: {}", serverUUID); + sourceConfig.setServerUUID(serverUUID); + } else { + log.warn("get source data source serverUUID empty please check"); + } + tableMgr = new RdbTableMgr(sourceConfig.getSourceConnectorConfig(), DatabaseConnection.sourceDataSource); canalServer = CanalServerWithEmbedded.instance(); @@ -152,9 +175,74 @@ protected void startEventParserInternal(CanalEventParser parser, boolean isGroup return instance; } }); - DatabaseConnection.sourceConfig = sourceConfig.getSourceConnectorConfig(); - DatabaseConnection.initSourceConnection(); - tableMgr = new RdbTableMgr(sourceConfig.getSourceConnectorConfig(), DatabaseConnection.sourceDataSource); + } + + private String queryServerUUID(DruidDataSource sourceDataSource, DataSourceType dataSourceType) { + String serverUUID = ""; + try { + String queryServerUUIDSql; + if (DataSourceType.MariaDB.equals(dataSourceType)) { + queryServerUUIDSql = SQL_SELECT_SERVER_UUID_IN_MARIADB; + } else { + queryServerUUIDSql = SQL_SHOW_SERVER_UUID_IN_MYSQL; + } + log.info("execute sql '{}' start.", queryServerUUIDSql); + try (PreparedStatement preparedStatement = sourceDataSource.getConnection().prepareStatement(queryServerUUIDSql)) { + ResultSet resultSet = preparedStatement.executeQuery(); + if (resultSet.next()) { + log.info("execute sql '{}' result:{}", queryServerUUIDSql, resultSet); + serverUUID = resultSet.getString("server_uuid"); + log.info("execute sql '{}',query server_uuid result:{}", queryServerUUIDSql, serverUUID); + return serverUUID; + } + } + } catch (Exception e) { + log.warn("select server_uuid failed,data source:{}", sourceDataSource, e); + throw new RuntimeException("select server_uuid failed"); + } + return serverUUID; + } + + // check is mariadb or mysql + private DataSourceType checkRDBDataSourceType(DruidDataSource sourceDataSource) { + try { + log.info("execute sql '{}' start.", SQL_SELECT_RDB_VERSION); + try (PreparedStatement preparedStatement = sourceDataSource.getConnection().prepareStatement(SQL_SELECT_RDB_VERSION)) { + ResultSet resultSet = preparedStatement.executeQuery(); + if (resultSet.next()) { + log.info("execute sql '{}' result:{}", SQL_SELECT_RDB_VERSION, resultSet); + String rdbVersion = resultSet.getString("rdb_version"); + if (StringUtils.isNotBlank(rdbVersion)) { + if (rdbVersion.toLowerCase().contains(DataSourceType.MariaDB.getName().toLowerCase())) { + return DataSourceType.MariaDB; + } + } + } + } + } catch (Exception e) { + log.warn("select rdb version failed,data source:{}", sourceDataSource, e); + throw new RuntimeException("select rdb version failed"); + } + return DataSourceType.MYSQL; + } + + private String buildTableFilters(CanalSourceIncrementConfig sourceConfig) { + StringBuilder tableFilterBuilder = new StringBuilder(); + Set dbDefinitions = sourceConfig.getSourceConnectorConfig().getDatabases(); + for (RdbDBDefinition dbDefinition : dbDefinitions) { + Set tableDefinitions = dbDefinition.getTables(); + for (RdbTableDefinition rdbTableDefinition : tableDefinitions) { + if (tableFilterBuilder.length() > 0) { + tableFilterBuilder.append(","); + } + String dbName = rdbTableDefinition.getSchemaName(); + String tableName = rdbTableDefinition.getTableName(); + tableFilterBuilder.append(dbName); + tableFilterBuilder.append("\\."); + tableFilterBuilder.append(tableName); + } + } + return tableFilterBuilder.toString(); } private Canal buildCanal(CanalSourceIncrementConfig sourceConfig) { @@ -254,14 +342,7 @@ public void start() throws Exception { @Override public void commit(ConnectRecord record) { - long batchId = Long.parseLong(record.getExtension("messageId")); - 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 @@ -362,10 +443,10 @@ public List poll() { result.add(connectRecord); } } - } else { - // for the message has been filtered need ack message - canalServer.ack(clientIdentity, message.getId()); + log.debug("message {} has been processed", message); } + log.debug("ack message, messageId {}", message.getId()); + canalServer.ack(clientIdentity, message.getId()); return result; } diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/position/CanalCheckPositionMgr.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/position/CanalCheckPositionMgr.java new file mode 100644 index 0000000000..149c62602c --- /dev/null +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/position/CanalCheckPositionMgr.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.connector.canal.source.position; + +import org.apache.eventmesh.common.AbstractComponent; +import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSourceCheckConfig; +import org.apache.eventmesh.common.config.connector.rdb.canal.JobRdbFullPosition; +import org.apache.eventmesh.common.config.connector.rdb.canal.RdbColumnDefinition; +import org.apache.eventmesh.common.config.connector.rdb.canal.RdbDBDefinition; +import org.apache.eventmesh.common.config.connector.rdb.canal.RdbTableDefinition; +import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.Constants; +import org.apache.eventmesh.common.config.connector.rdb.canal.mysql.MySQLTableDef; +import org.apache.eventmesh.common.remote.offset.RecordPosition; +import org.apache.eventmesh.common.remote.offset.canal.CanalFullRecordOffset; +import org.apache.eventmesh.common.utils.JsonUtils; +import org.apache.eventmesh.connector.canal.DatabaseConnection; +import org.apache.eventmesh.connector.canal.source.table.RdbSimpleTable; +import org.apache.eventmesh.connector.canal.source.table.RdbTableMgr; + +import org.apache.commons.lang3.StringUtils; + +import java.sql.JDBCType; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.LinkedHashMap; +import java.util.Map; + +import javax.sql.DataSource; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class CanalCheckPositionMgr extends AbstractComponent { + + private final CanalSourceCheckConfig config; + private final Map positions = new LinkedHashMap<>(); + private final RdbTableMgr tableMgr; + + public CanalCheckPositionMgr(CanalSourceCheckConfig config, RdbTableMgr tableMgr) { + this.config = config; + this.tableMgr = tableMgr; + } + + @Override + protected void run() throws Exception { + if (config == null || config.getSourceConnectorConfig() == null || config.getSourceConnectorConfig().getDatabases() == null) { + log.info("config or database is null"); + return; + } + prepareRecordPosition(); + initPositions(); + } + + public void prepareRecordPosition() { + if (config.getStartPosition() != null && !config.getStartPosition().isEmpty()) { + for (RecordPosition record : config.getStartPosition()) { + CanalFullRecordOffset offset = (CanalFullRecordOffset) record.getRecordOffset(); + RdbSimpleTable table = new RdbSimpleTable(offset.getPosition().getSchema(), offset.getPosition().getTableName()); + positions.put(table, offset.getPosition()); + } + } + } + + public JobRdbFullPosition getPosition(RdbSimpleTable table) { + return positions.get(table); + } + + public boolean isFinished() { + for (JobRdbFullPosition position : positions.values()) { + if (!position.isFinished()) { + log.info("schema [{}] table [{}] is not finish", position.getSchema(), position.getTableName()); + return false; + } + } + return true; + } + + private void initPositions() { + for (RdbDBDefinition database : config.getSourceConnectorConfig().getDatabases()) { + for (RdbTableDefinition table : database.getTables()) { + try { + RdbSimpleTable simpleTable = new RdbSimpleTable(database.getSchemaName(), table.getTableName()); + RdbTableDefinition tableDefinition; + if ((tableDefinition = tableMgr.getTable(simpleTable)) == null) { + log.error("db [{}] table [{}] definition is null", database.getSchemaName(), table.getTableName()); + continue; + } + log.info("init position of data [{}] table [{}]", database.getSchemaName(), table.getTableName()); + + JobRdbFullPosition recordPosition = positions.get(simpleTable); + if (recordPosition == null || !recordPosition.isFinished()) { + positions.put(simpleTable, + fetchTableInfo(DatabaseConnection.sourceDataSource, (MySQLTableDef) tableDefinition, recordPosition)); + } + } catch (Exception e) { + log.error("process schema [{}] table [{}] position fail", database.getSchemaName(), table.getTableName(), e); + } + + } + } + } + + private JobRdbFullPosition fetchTableInfo(DataSource dataSource, MySQLTableDef tableDefinition, JobRdbFullPosition recordPosition) + throws SQLException { + TableFullPosition position = new TableFullPosition(); + Map preMinPrimaryKeys = new LinkedHashMap<>(); + Map preMaxPrimaryKeys = new LinkedHashMap<>(); + for (String pk : tableDefinition.getPrimaryKeys()) { + Object min = fetchMinPrimaryKey(dataSource, tableDefinition, preMinPrimaryKeys, pk); + Object max = fetchMaxPrimaryKey(dataSource, tableDefinition, preMaxPrimaryKeys, pk); + preMinPrimaryKeys.put(pk, min); + preMaxPrimaryKeys.put(pk, max); + position.getCurPrimaryKeyCols().put(pk, min); + position.getMinPrimaryKeyCols().put(pk, min); + position.getMaxPrimaryKeyCols().put(pk, max); + } + JobRdbFullPosition jobRdbFullPosition = new JobRdbFullPosition(); + if (recordPosition != null) { + if (StringUtils.isNotBlank(recordPosition.getPrimaryKeyRecords())) { + TableFullPosition record = JsonUtils.parseObject(recordPosition.getPrimaryKeyRecords(), TableFullPosition.class); + if (record != null && record.getCurPrimaryKeyCols() != null && !record.getCurPrimaryKeyCols().isEmpty()) { + position.setCurPrimaryKeyCols(record.getCurPrimaryKeyCols()); + } + } + jobRdbFullPosition.setPercent(recordPosition.getPercent()); + } + long rowCount = queryCurTableRowCount(dataSource, tableDefinition); + jobRdbFullPosition.setSchema(tableDefinition.getSchemaName()); + jobRdbFullPosition.setTableName(tableDefinition.getTableName()); + jobRdbFullPosition.setMaxCount(rowCount); + jobRdbFullPosition.setPrimaryKeyRecords(JsonUtils.toJSONString(position)); + return jobRdbFullPosition; + } + + + private long queryCurTableRowCount(DataSource datasource, MySQLTableDef tableDefinition) throws SQLException { + String sql = "select `AVG_ROW_LENGTH`,`DATA_LENGTH` from information_schema.TABLES where `TABLE_SCHEMA`='" + tableDefinition.getSchemaName() + + "' and `TABLE_NAME`='" + tableDefinition.getTableName() + "'"; + try (Statement statement = datasource.getConnection().createStatement(); ResultSet resultSet = statement.executeQuery(sql)) { + long result = 0L; + if (resultSet.next()) { + long avgRowLength = resultSet.getLong("AVG_ROW_LENGTH"); + long dataLength = resultSet.getLong("DATA_LENGTH"); + if (avgRowLength != 0L) { + result = dataLength / avgRowLength; + } + } + return result; + } + } + + private void appendPrePrimaryKey(Map preMap, StringBuilder sql) { + if (preMap != null && !preMap.isEmpty()) { + sql.append(" WHERE "); + boolean first = true; + for (Map.Entry entry : preMap.entrySet()) { + if (first) { + first = false; + } else { + sql.append(" AND "); + } + sql.append(Constants.MySQLQuot).append(entry.getKey()).append(Constants.MySQLQuot).append("=?"); + } + } + } + + private void setValue2Statement(PreparedStatement ps, Map preMap, MySQLTableDef tableDefinition) throws SQLException { + if (preMap != null && !preMap.isEmpty()) { + int index = 1; + for (Map.Entry entry : preMap.entrySet()) { + RdbColumnDefinition def = tableDefinition.getColumnDefinitions().get(entry.getKey()); + ps.setObject(index, entry.getValue(), def.getJdbcType().getVendorTypeNumber()); + ++index; + } + } + } + + private Object fetchMinPrimaryKey(DataSource dataSource, MySQLTableDef tableDefinition, Map prePrimary, String curPrimaryKeyCol) + throws SQLException { + StringBuilder builder = new StringBuilder(); + builder.append("SELECT MIN(").append(Constants.MySQLQuot).append(curPrimaryKeyCol).append(Constants.MySQLQuot) + .append(") min_primary_key FROM").append(Constants.MySQLQuot).append(tableDefinition.getSchemaName()).append(Constants.MySQLQuot) + .append(".").append(Constants.MySQLQuot).append(tableDefinition.getTableName()).append(Constants.MySQLQuot); + appendPrePrimaryKey(prePrimary, builder); + String sql = builder.toString(); + log.info("fetch min primary sql [{}]", sql); + try (PreparedStatement statement = dataSource.getConnection().prepareStatement(sql)) { + setValue2Statement(statement, prePrimary, tableDefinition); + try (ResultSet resultSet = statement.executeQuery()) { + if (resultSet.next()) { + RdbColumnDefinition columnDefinition = tableDefinition.getColumnDefinitions().get(curPrimaryKeyCol); + if (columnDefinition.getJdbcType() == JDBCType.TIMESTAMP) { + return resultSet.getString("min_primary_key"); + } else { + return resultSet.getObject("min_primary_key"); + } + } + } + } + return null; + } + + private Object fetchMaxPrimaryKey(DataSource dataSource, MySQLTableDef tableDefinition, Map prePrimary, String curPrimaryKeyCol) + throws SQLException { + StringBuilder builder = new StringBuilder(); + builder.append("SELECT MAX(").append(Constants.MySQLQuot).append(curPrimaryKeyCol).append(Constants.MySQLQuot) + .append(") max_primary_key FROM").append(Constants.MySQLQuot).append(tableDefinition.getSchemaName()).append(Constants.MySQLQuot) + .append(".").append(Constants.MySQLQuot).append(tableDefinition.getTableName()).append(Constants.MySQLQuot); + appendPrePrimaryKey(prePrimary, builder); + String sql = builder.toString(); + log.info("fetch max primary sql [{}]", sql); + try (PreparedStatement statement = dataSource.getConnection().prepareStatement(sql)) { + setValue2Statement(statement, prePrimary, tableDefinition); + try (ResultSet resultSet = statement.executeQuery()) { + if (resultSet.next()) { + RdbColumnDefinition columnDefinition = tableDefinition.getColumnDefinitions().get(curPrimaryKeyCol); + if (columnDefinition.getJdbcType() == JDBCType.TIMESTAMP) { + return resultSet.getString("max_primary_key"); + } else { + return resultSet.getObject("max_primary_key"); + } + } + } + } + return null; + } + + + @Override + protected void shutdown() throws Exception { + + } +} diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/position/CanalFullPositionMgr.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/position/CanalFullPositionMgr.java index 0ae1f8f8ff..dad0ddbf3b 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/position/CanalFullPositionMgr.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/position/CanalFullPositionMgr.java @@ -34,6 +34,7 @@ import org.apache.commons.lang3.StringUtils; +import java.sql.Connection; import java.sql.JDBCType; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -153,7 +154,8 @@ private JobRdbFullPosition fetchTableInfo(DataSource dataSource, MySQLTableDef t private long queryCurTableRowCount(DataSource datasource, MySQLTableDef tableDefinition) throws SQLException { String sql = "select `AVG_ROW_LENGTH`,`DATA_LENGTH` from information_schema.TABLES where `TABLE_SCHEMA`='" + tableDefinition.getSchemaName() + "' and `TABLE_NAME`='" + tableDefinition.getTableName() + "'"; - try (Statement statement = datasource.getConnection().createStatement(); ResultSet resultSet = statement.executeQuery(sql)) { + try (Connection conn = datasource.getConnection(); Statement statement = conn.createStatement(); + ResultSet resultSet = statement.executeQuery(sql)) { long result = 0L; if (resultSet.next()) { long avgRowLength = resultSet.getLong("AVG_ROW_LENGTH"); @@ -201,7 +203,7 @@ private Object fetchMinPrimaryKey(DataSource dataSource, MySQLTableDef tableDefi appendPrePrimaryKey(prePrimary, builder); String sql = builder.toString(); log.info("fetch min primary sql [{}]", sql); - try (PreparedStatement statement = dataSource.getConnection().prepareStatement(sql)) { + try (Connection conn = dataSource.getConnection(); PreparedStatement statement = conn.prepareStatement(sql)) { setValue2Statement(statement, prePrimary, tableDefinition); try (ResultSet resultSet = statement.executeQuery()) { if (resultSet.next()) { @@ -226,7 +228,7 @@ private Object fetchMaxPrimaryKey(DataSource dataSource, MySQLTableDef tableDefi appendPrePrimaryKey(prePrimary, builder); String sql = builder.toString(); log.info("fetch max primary sql [{}]", sql); - try (PreparedStatement statement = dataSource.getConnection().prepareStatement(sql)) { + try (Connection conn = dataSource.getConnection(); PreparedStatement statement = conn.prepareStatement(sql)) { setValue2Statement(statement, prePrimary, tableDefinition); try (ResultSet resultSet = statement.executeQuery()) { if (resultSet.next()) { diff --git a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/table/RdbTableMgr.java b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/table/RdbTableMgr.java index de7a45dc99..954b81ca70 100644 --- a/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/table/RdbTableMgr.java +++ b/eventmesh-connectors/eventmesh-connector-canal/src/main/java/org/apache/eventmesh/connector/canal/source/table/RdbTableMgr.java @@ -27,13 +27,13 @@ import org.apache.eventmesh.common.exception.EventMeshException; import org.apache.eventmesh.connector.canal.SqlUtils; +import java.sql.Connection; import java.sql.JDBCType; import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; @@ -49,6 +49,7 @@ @Slf4j public class RdbTableMgr extends AbstractComponent { + private final JdbcConfig config; private final Map tables = new HashMap<>(); private final DataSource dataSource; @@ -85,7 +86,7 @@ protected void run() { if (primaryKeys == null || primaryKeys.isEmpty() || primaryKeys.get(table.getTableName()) == null) { log.warn("init db [{}] table [{}] info, and primary keys are empty", db.getSchemaName(), table.getTableName()); } else { - mysqlTable.setPrimaryKeys(new HashSet<>(primaryKeys.get(table.getTableName()))); + mysqlTable.setPrimaryKeys(primaryKeys.get(table.getTableName())); } if (columns == null || columns.isEmpty() || columns.get(table.getTableName()) == null) { log.warn("init db [{}] table [{}] info, and columns are empty", db.getSchemaName(), table.getTableName()); @@ -116,25 +117,26 @@ private Map> queryTablePrimaryKey(String schema, List { - if (v == null) { - v = new LinkedList<>(); - } - v.add(colName); - return v; - }); + try (ResultSet rs = statement.executeQuery()) { + if (rs == null) { + return null; + } + while (rs.next()) { + String tableName = rs.getString("TABLE_NAME"); + String colName = rs.getString("COLUMN_NAME"); + primaryKeys.compute(tableName, (k, v) -> { + if (v == null) { + v = new LinkedList<>(); + } + v.add(colName); + return v; + }); + } } - resultSet.close(); } return primaryKeys; } @@ -146,22 +148,27 @@ private Map> queryColumns(String schema, List> cols = new LinkedHashMap<>(); - try (PreparedStatement statement = dataSource.getConnection().prepareStatement(sql)) { + Connection conn = null; + PreparedStatement statement = null; + ResultSet rs = null; + try { + conn = dataSource.getConnection(); + statement = conn.prepareStatement(sql); statement.setString(1, schema); SqlUtils.setInClauseParameters(statement, 2, tables); - ResultSet resultSet = statement.executeQuery(); - if (resultSet == null) { + rs = statement.executeQuery(); + if (rs == null) { return null; } - while (resultSet.next()) { - String dataType = resultSet.getString("DATA_TYPE"); + while (rs.next()) { + String dataType = rs.getString("DATA_TYPE"); JDBCType jdbcType = SqlUtils.toJDBCType(dataType); MySQLColumnDef col = new MySQLColumnDef(); col.setJdbcType(jdbcType); col.setType(CanalMySQLType.valueOfCode(dataType)); - String colName = resultSet.getString("COLUMN_NAME"); + String colName = rs.getString("COLUMN_NAME"); col.setName(colName); - String tableName = resultSet.getString("TABLE_NAME"); + String tableName = rs.getString("TABLE_NAME"); cols.compute(tableName, (k, v) -> { if (v == null) { v = new LinkedList<>(); @@ -170,7 +177,30 @@ private Map> queryColumns(String schema, List Date: Tue, 10 Dec 2024 22:06:16 +0800 Subject: [PATCH 4/5] [ISSUE #5141] update eventmesh-admin-server module --- eventmesh-admin-server/bin/stop-admin.sh | 88 ++++++ eventmesh-admin-server/conf/application.yaml | 24 +- eventmesh-admin-server/conf/eventmesh.sql | 33 ++ eventmesh-admin-server/conf/log4j2.xml | 68 ----- .../conf/mapper/EventMeshMonitorMapper.xml | 46 +++ .../admin/server/AdminServerProperties.java | 3 +- .../admin/server/ExampleAdminServer.java | 9 +- .../AdminServerConstants.java | 2 +- .../admin/server/web/HttpServer.java | 128 +++++++- .../server/web/config/MybatisPlusConfig.java | 39 +++ .../admin/server/web/db/DruidDataSource.java | 150 +++++++++ .../web/db/entity/EventMeshMonitor.java | 52 ++++ .../db/entity/EventMeshWeredisPosition.java | 59 ++++ .../web/db/mapper/EventMeshMonitorMapper.java | 37 +++ .../db/service/EventMeshMonitorService.java | 29 ++ .../db/service/EventMeshTaskInfoService.java | 8 + .../impl/EventMeshMonitorServiceImpl.java | 39 +++ .../impl/EventMeshTaskInfoServiceImpl.java | 288 +++++++++++++++++- .../handler/impl/ReportJobRequestHandler.java | 47 ++- .../handler/impl/ReportMonitorHandler.java | 97 ++++++ .../web/handler/impl/ReportVerifyHandler.java | 3 +- .../admin/server/web/pojo/BinlogPosition.java | 27 ++ .../admin/server/web/pojo/TaskDetail.java | 10 + .../web/service/job/JobInfoBizService.java | 43 ++- .../service/monitor/MonitorBizService.java | 111 +++++++ .../position/IRecordPositionHandler.java | 30 ++ .../service/position/PositionBizService.java | 7 + .../web/service/position/PositionHandler.java | 2 +- .../position/impl/HttpPositionHandler.java | 6 + .../position/impl/MysqlPositionHandler.java | 177 ++++++++++- .../web/service/task/TaskBizService.java | 191 +++++++++++- .../admin/server/web/utils/Base64.java | 125 ++++++++ .../admin/server/web/utils/Base64Utils.java | 94 ++++++ .../admin/server/web/utils/EncryptUtil.java | 138 +++++++++ .../admin/server/web/utils/JdbcUtils.java | 43 +++ .../admin/server/web/utils/ParamType.java | 26 ++ .../admin/server/web/utils/RSAUtils.java | 255 ++++++++++++++++ .../eventmesh/common/remote/JobState.java | 2 +- .../remote/request/QueryTaskInfoRequest.java | 49 +++ .../request/QueryTaskMonitorRequest.java | 31 ++ .../remote/request/RecordPositionRequest.java | 41 +++ .../remote/request/TaskBachRequest.java | 29 ++ .../common/remote/request/TaskIDRequest.java | 27 ++ .../remote/response/BaseRemoteResponse.java | 4 +- .../remote/response/CreateTaskResponse.java | 2 +- .../remote/response/HttpResponseResult.java | 65 ++++ .../response/QueryTaskInfoResponse.java | 152 +++++++++ .../response/QueryTaskMonitorResponse.java | 31 ++ .../common/remote/task/TaskMonitor.java | 40 +++ 49 files changed, 2894 insertions(+), 113 deletions(-) create mode 100644 eventmesh-admin-server/bin/stop-admin.sh create mode 100644 eventmesh-admin-server/conf/mapper/EventMeshMonitorMapper.xml rename eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/{constatns => constants}/AdminServerConstants.java (95%) create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/config/MybatisPlusConfig.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/DruidDataSource.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshMonitor.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshWeredisPosition.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/mapper/EventMeshMonitorMapper.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/EventMeshMonitorService.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshMonitorServiceImpl.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportMonitorHandler.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/BinlogPosition.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/monitor/MonitorBizService.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/IRecordPositionHandler.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/Base64.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/Base64Utils.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/EncryptUtil.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/JdbcUtils.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/ParamType.java create mode 100644 eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/RSAUtils.java create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/QueryTaskInfoRequest.java create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/QueryTaskMonitorRequest.java create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/RecordPositionRequest.java create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/TaskBachRequest.java create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/TaskIDRequest.java create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/HttpResponseResult.java create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/QueryTaskInfoResponse.java create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/QueryTaskMonitorResponse.java create mode 100644 eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/task/TaskMonitor.java diff --git a/eventmesh-admin-server/bin/stop-admin.sh b/eventmesh-admin-server/bin/stop-admin.sh new file mode 100644 index 0000000000..207531d7fa --- /dev/null +++ b/eventmesh-admin-server/bin/stop-admin.sh @@ -0,0 +1,88 @@ +#!/bin/bash +# +# Licensed to 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. Apache Software Foundation (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. + +# Detect operating system +OS=$(uname) + +EVENTMESH_ADMIN_HOME=`cd $(dirname $0)/.. && pwd` + +export EVENTMESH_ADMIN_HOME + +function get_pid { + local ppid="" + if [ -f ${EVENTMESH_ADMIN_HOME}/bin/pid-admin.file ]; then + ppid=$(cat ${EVENTMESH_ADMIN_HOME}/bin/pid-admin.file) + # If the process does not exist, it indicates that the previous process terminated abnormally. + if [ ! -d /proc/$ppid ]; then + # Remove the residual file and return an error status. + rm ${EVENTMESH_ADMIN_HOME}/bin/pid-admin.file + echo -e "ERROR\t EventMesh admin process had already terminated unexpectedly before, please check log output." + ppid="" + fi + else + if [[ $OS =~ Msys ]]; then + # There is a Bug on Msys that may not be able to kill the identified process + ppid=`jps -v | grep -i "org.apache.eventmesh.admin.server.ExampleAdminServer" | grep java | grep -v grep | awk -F ' ' {'print $1'}` + elif [[ $OS =~ Darwin ]]; then + # Known problem: grep Java may not be able to accurately identify Java processes + ppid=$(/bin/ps -o user,pid,command | grep "java" | grep -i "org.apache.eventmesh.admin.server.ExampleAdminServer" | grep -Ev "^root" |awk -F ' ' {'print $2'}) + else + # It is required to identify the process as accurately as possible on Linux + ppid=$(ps -C java -o user,pid,command --cols 99999 | grep -w $EVENTMESH_ADMIN_HOME | grep -i "org.apache.eventmesh.admin.server.ExampleAdminServer" | grep -Ev "^root" |awk -F ' ' {'print $2'}) + fi + fi + echo "$ppid"; +} + +pid=$(get_pid) +if [[ $pid == "ERROR"* ]]; then + echo -e "${pid}" + exit 9 +fi +if [ -z "$pid" ];then + echo -e "ERROR\t No EventMesh admin server running." + exit 9 +fi + +kill ${pid} +echo "Send shutdown request to EventMesh admin(${pid}) OK" + +[[ $OS =~ Msys ]] && PS_PARAM=" -W " +stop_timeout=60 +for no in $(seq 1 $stop_timeout); do + if ps $PS_PARAM -p "$pid" 2>&1 > /dev/null; then + if [ $no -lt $stop_timeout ]; then + echo "[$no] server shutting down ..." + sleep 1 + continue + fi + + echo "shutdown server timeout, kill process: $pid" + kill -9 $pid; sleep 1; break; + echo "`date +'%Y-%m-%-d %H:%M:%S'` , pid : [$pid] , error message : abnormal shutdown which can not be closed within 60s" > ../logs/shutdown.error + else + echo "shutdown server ok!"; break; + fi +done + +if [ -f "pid-admin.file" ]; then + rm pid-admin.file +fi + + diff --git a/eventmesh-admin-server/conf/application.yaml b/eventmesh-admin-server/conf/application.yaml index 3d702e579e..7765d90ce8 100644 --- a/eventmesh-admin-server/conf/application.yaml +++ b/eventmesh-admin-server/conf/application.yaml @@ -21,6 +21,24 @@ spring: username: //db_username password: //db_password driver-class-name: com.mysql.cj.jdbc.Driver + initialSize: 1 + minIdle: 1 + maxActive: 20 + maxWait: 10000 + timeBetweenEvictionRunsMillis: 60000 + minEvictableIdleTimeMillis: 300000 + validationQuery: SELECT 1 FROM DUAL + testWhileIdle: true + testOnBorrow: false + testOnReturn: false + poolPreparedStatements: false + maxPoolPreparedStatementPerConnectionSize: 20 + filters: stat + connectionProperties: "druid.stat.mergeSql=true;druid.stat.slowSqlMillis=5000" +# secret keys +sysPubKey: +appPrivKey: + mybatis-plus: mapper-locations: classpath:mapper/*.xml configuration: @@ -35,8 +53,6 @@ event-mesh: # grpc server port port: 8081 adminServerList: - R1: - - http://localhost:8082 - R2: - - http://localhost:8082 + R1: http://localhost:8082;http://localhost:8082 + R2: http://localhost:8092;http://localhost:8092 region: R1 \ No newline at end of file diff --git a/eventmesh-admin-server/conf/eventmesh.sql b/eventmesh-admin-server/conf/eventmesh.sql index 6e28daca8a..4d11ab1585 100644 --- a/eventmesh-admin-server/conf/eventmesh.sql +++ b/eventmesh-admin-server/conf/eventmesh.sql @@ -146,6 +146,39 @@ CREATE TABLE IF NOT EXISTS `event_mesh_verify` ( PRIMARY KEY (`id`) ) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; +-- eventmesh.event_mesh_weredis_position definition +CREATE TABLE `event_mesh_weredis_position` ( + `id` int(10) unsigned NOT NULL AUTO_INCREMENT, + `jobID` varchar(50) COLLATE utf8_bin NOT NULL DEFAULT '', + `address` varchar(50) COLLATE utf8_bin DEFAULT NULL, + `clusterName` varchar(50) COLLATE utf8_bin DEFAULT NULL, + `partitionName` varchar(50) COLLATE utf8_bin DEFAULT NULL, + `masterReplid` varchar(50) COLLATE utf8_bin DEFAULT NULL, + `host` varchar(50) COLLATE utf8_bin DEFAULT NULL, + `replOffset` bigint(20) NOT NULL DEFAULT '-1', + `createTime` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, + `updateTime` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, + PRIMARY KEY (`id`), + UNIQUE KEY `jobID` (`jobID`) +) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8 COLLATE=utf8_bin ROW_FORMAT=DYNAMIC; + + +CREATE TABLE `event_mesh_monitor` ( + `id` int(10) unsigned NOT NULL AUTO_INCREMENT, + `taskID` varchar(50) COLLATE utf8_bin DEFAULT NULL, + `jobID` varchar(50) COLLATE utf8_bin DEFAULT NULL, + `address` varchar(50) COLLATE utf8_bin DEFAULT NULL, + `transportType` varchar(50) COLLATE utf8_bin DEFAULT NULL, + `connectorStage` varchar(50) COLLATE utf8_bin DEFAULT NULL, + `totalReqNum` bigint DEFAULT NULL, + `totalTimeCost` bigint DEFAULT NULL, + `maxTimeCost` bigint DEFAULT NULL, + `avgTimeCost` bigint DEFAULT NULL, + `tps` double DEFAULT NULL, + `createTime` timestamp NULL DEFAULT CURRENT_TIMESTAMP, + PRIMARY KEY (`id`) +) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8 COLLATE=utf8_bin; + /*!40101 SET SQL_MODE=IFNULL(@OLD_SQL_MODE, '') */; /*!40014 SET FOREIGN_KEY_CHECKS=IFNULL(@OLD_FOREIGN_KEY_CHECKS, 1) */; /*!40101 SET CHARACTER_SET_CLIENT=@OLD_CHARACTER_SET_CLIENT */; diff --git a/eventmesh-admin-server/conf/log4j2.xml b/eventmesh-admin-server/conf/log4j2.xml index 6341a0e629..acc6acb8ba 100644 --- a/eventmesh-admin-server/conf/log4j2.xml +++ b/eventmesh-admin-server/conf/log4j2.xml @@ -28,74 +28,6 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/eventmesh-admin-server/conf/mapper/EventMeshMonitorMapper.xml b/eventmesh-admin-server/conf/mapper/EventMeshMonitorMapper.xml new file mode 100644 index 0000000000..f77fb8ba77 --- /dev/null +++ b/eventmesh-admin-server/conf/mapper/EventMeshMonitorMapper.xml @@ -0,0 +1,46 @@ + + + + + + + + + + + + + + + + + + + + + + + + id,taskID,jobID,address,transportType,connectorStage, + totalReqNum,totalTimeCost,maxTimeCost,avgTimeCost, + tps,createTime + + 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 612d398078..2e6d3c018a 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,7 +17,6 @@ package org.apache.eventmesh.admin.server; -import java.util.List; import java.util.Map; import org.springframework.boot.context.properties.ConfigurationProperties; @@ -35,6 +34,6 @@ public class AdminServerProperties { private String configurationPath; private String configurationFile; private String serviceName; - private Map> adminServerList; + private Map adminServerList; private String region; } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/ExampleAdminServer.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/ExampleAdminServer.java index b179a790c5..d5c52f58bc 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/ExampleAdminServer.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/ExampleAdminServer.java @@ -17,17 +17,22 @@ package org.apache.eventmesh.admin.server; -import org.apache.eventmesh.admin.server.constatns.AdminServerConstants; +import org.apache.eventmesh.admin.server.constants.AdminServerConstants; import org.apache.eventmesh.common.config.ConfigService; import org.springframework.boot.SpringApplication; import org.springframework.boot.autoconfigure.SpringBootApplication; +import org.springframework.boot.autoconfigure.jdbc.DataSourceAutoConfiguration; -@SpringBootApplication(scanBasePackages = "org.apache.eventmesh.admin.server") +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@SpringBootApplication(scanBasePackages = "org.apache.eventmesh.admin.server", exclude = {DataSourceAutoConfiguration.class}) public class ExampleAdminServer { public static void main(String[] args) throws Exception { ConfigService.getInstance().setConfigPath(AdminServerConstants.EVENTMESH_CONF_HOME).setRootConfig(AdminServerConstants.EVENTMESH_CONF_FILE); SpringApplication.run(ExampleAdminServer.class); + log.info("wedts-admin start success."); } } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/constatns/AdminServerConstants.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/constants/AdminServerConstants.java similarity index 95% rename from eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/constatns/AdminServerConstants.java rename to eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/constants/AdminServerConstants.java index 44afaca1c2..8ed079fd31 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/constatns/AdminServerConstants.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/constants/AdminServerConstants.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.eventmesh.admin.server.constatns; +package org.apache.eventmesh.admin.server.constants; public class AdminServerConstants { public static final String CONF_ENV = "configurationPath"; 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 2454e9f02c..0a20d8645e 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 @@ -17,15 +17,28 @@ package org.apache.eventmesh.admin.server.web; +import org.apache.eventmesh.admin.server.web.db.service.EventMeshTaskInfoService; +import org.apache.eventmesh.admin.server.web.service.monitor.MonitorBizService; import org.apache.eventmesh.admin.server.web.service.task.TaskBizService; import org.apache.eventmesh.admin.server.web.service.verify.VerifyBizService; import org.apache.eventmesh.common.remote.request.CreateTaskRequest; +import org.apache.eventmesh.common.remote.request.QueryTaskInfoRequest; +import org.apache.eventmesh.common.remote.request.QueryTaskMonitorRequest; +import org.apache.eventmesh.common.remote.request.ReportMonitorRequest; import org.apache.eventmesh.common.remote.request.ReportVerifyRequest; +import org.apache.eventmesh.common.remote.request.TaskBachRequest; +import org.apache.eventmesh.common.remote.request.TaskIDRequest; import org.apache.eventmesh.common.remote.response.CreateTaskResponse; +import org.apache.eventmesh.common.remote.response.HttpResponseResult; +import org.apache.eventmesh.common.remote.response.QueryTaskInfoResponse; +import org.apache.eventmesh.common.remote.response.QueryTaskMonitorResponse; +import org.apache.eventmesh.common.remote.response.SimpleResponse; import org.apache.eventmesh.common.utils.JsonUtils; +import java.util.ArrayList; +import java.util.List; + import org.springframework.beans.factory.annotation.Autowired; -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; @@ -44,30 +57,127 @@ public class HttpServer { @Autowired private VerifyBizService verifyService; + @Autowired + private MonitorBizService monitorService; + + @Autowired + private EventMeshTaskInfoService taskInfoService; + @RequestMapping(value = "/createTask", method = RequestMethod.POST) - public ResponseEntity createOrUpdateTask(@RequestBody CreateTaskRequest task) { + public String createOrUpdateTask(@RequestBody CreateTaskRequest task) { log.info("receive http proto create task:{}", task); CreateTaskResponse createTaskResponse = taskService.createTask(task); log.info("receive http proto create task result:{}", createTaskResponse); - return ResponseEntity.ok(JsonUtils.toJSONString(Response.success(createTaskResponse))); + SimpleResponse simpleResponse = new SimpleResponse(); + simpleResponse.setData(createTaskResponse); + return JsonUtils.toJSONString(simpleResponse); } @RequestMapping(value = "/reportVerify", method = RequestMethod.POST) - public ResponseEntity reportVerify(@RequestBody ReportVerifyRequest request) { + public String reportVerify(@RequestBody ReportVerifyRequest request) { log.info("receive http proto report verify request:{}", request); boolean result = verifyService.reportVerifyRecord(request); log.info("receive http proto report verify result:{}", result); + SimpleResponse simpleResponse = new SimpleResponse(); + simpleResponse.setData(result); + return JsonUtils.toJSONString(simpleResponse); + } + + @RequestMapping(value = "/reportMonitor", method = RequestMethod.POST) + public String reportMonitor(@RequestBody ReportMonitorRequest request) { + log.info("receive http proto report monitor request:{}", request); + boolean result = monitorService.reportMonitorRecord(request); + log.info("receive http proto report monitor result:{}", result); + SimpleResponse simpleResponse = new SimpleResponse(); + simpleResponse.setData(result); + return JsonUtils.toJSONString(simpleResponse); + } + + @RequestMapping(value = "/queryTaskMonitor", method = RequestMethod.POST) + public String queryTaskMonitor(@RequestBody QueryTaskMonitorRequest request) { + log.info("receive http proto query task monitor request:{}", request); + QueryTaskMonitorResponse result = monitorService.queryTaskMonitors(request); + log.info("receive http proto query task monitor result:{}", result); + SimpleResponse simpleResponse = new SimpleResponse(); + simpleResponse.setData(result); + return JsonUtils.toJSONString(simpleResponse); + } + + @RequestMapping(value = "/queryTaskInfo", method = RequestMethod.POST) + public HttpResponseResult queryTaskInfo(@RequestBody QueryTaskInfoRequest taskInfoRequest) { + log.info("receive http query task info request:{}", taskInfoRequest); + List taskInfosResponse = taskService.queryTaskInfo(taskInfoRequest); + log.info("receive http query task info taskInfosResponse:{}", taskInfoRequest); + if (taskInfosResponse.isEmpty()) { + return HttpResponseResult.failed("NOT FOUND"); + } + return HttpResponseResult.success(taskInfosResponse); + } + + @RequestMapping(value = "/deleteTask", method = RequestMethod.DELETE) + public HttpResponseResult deleteTask(@RequestBody TaskIDRequest taskIDRequest) { + log.info("receive need to delete taskID:{}", taskIDRequest.getTaskID()); + boolean result = taskService.deleteTaskByTaskID(taskIDRequest); if (result) { - return ResponseEntity.ok("report verify success.request:" + JsonUtils.toJSONString(request)); + return HttpResponseResult.success(); } else { - return ResponseEntity.internalServerError().body("report verify success.request:" + JsonUtils.toJSONString(request)); + return HttpResponseResult.failed(); } } - public boolean deleteTask(Long id) { - return false; + @RequestMapping(value = "/startTask", method = RequestMethod.POST) + public HttpResponseResult startTask(@RequestBody TaskIDRequest taskIDRequest) { + log.info("receive start task ID:{}", taskIDRequest.getTaskID()); + taskService.startTask(taskIDRequest); + return HttpResponseResult.success(); + } + + @RequestMapping(value = "/restartTask", method = RequestMethod.POST) + public HttpResponseResult restartTask(@RequestBody TaskIDRequest taskIDRequest) { + log.info("receive restart task ID:{}", taskIDRequest.getTaskID()); + taskService.restartTask(taskIDRequest); + return HttpResponseResult.success(); } + @RequestMapping(value = "/stopTask", method = RequestMethod.POST) + public HttpResponseResult stopTask(@RequestBody TaskIDRequest taskIDRequest) { + log.info("receive stop task ID:{}", taskIDRequest.getTaskID()); + taskService.stopTask(taskIDRequest); + return HttpResponseResult.success(); + } + + @RequestMapping(value = "/restartBatch", method = RequestMethod.POST) + public HttpResponseResult restartBatch(@RequestBody List taskBachRequestList) { + log.info("receive restart batch task IDs:{}", taskBachRequestList); + List errorNames = new ArrayList<>(); + taskService.restartBatchTask(taskBachRequestList, errorNames); + if (!errorNames.isEmpty()) { + return HttpResponseResult.failed(errorNames); + } + return HttpResponseResult.success(); + } + + @RequestMapping(value = "stopBatch", method = RequestMethod.POST) + public HttpResponseResult stopBatch(@RequestBody List taskBachRequestList) { + log.info("receive stop batch task IDs:{}", taskBachRequestList); + List errorNames = new ArrayList<>(); + taskService.stopBatchTask(taskBachRequestList, errorNames); + if (!errorNames.isEmpty()) { + return HttpResponseResult.failed(errorNames); + } + return HttpResponseResult.success(); + } + + @RequestMapping(value = "/startBatch", method = RequestMethod.POST) + public HttpResponseResult startBatch(@RequestBody List taskBachRequestList) { + log.info("receive start batch task IDs:{}", taskBachRequestList); + List errorNames = new ArrayList<>(); + taskService.startBatchTask(taskBachRequestList, errorNames); + if (!errorNames.isEmpty()) { + return HttpResponseResult.failed(errorNames); + } + return HttpResponseResult.success(); + } -} +} \ No newline at end of file diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/config/MybatisPlusConfig.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/config/MybatisPlusConfig.java new file mode 100644 index 0000000000..15d362bcd0 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/config/MybatisPlusConfig.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.config; + +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; + +import com.baomidou.mybatisplus.annotation.DbType; +import com.baomidou.mybatisplus.extension.plugins.MybatisPlusInterceptor; +import com.baomidou.mybatisplus.extension.plugins.inner.PaginationInnerInterceptor; + +@Configuration +public class MybatisPlusConfig { + + @Bean + public MybatisPlusInterceptor paginationInterceptor() { + MybatisPlusInterceptor interceptor = new MybatisPlusInterceptor(); + PaginationInnerInterceptor pageInterceptor = new PaginationInnerInterceptor(DbType.MYSQL); + pageInterceptor.setMaxLimit(500L); + interceptor.addInnerInterceptor(pageInterceptor); + return interceptor; + } + +} \ No newline at end of file diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/DruidDataSource.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/DruidDataSource.java new file mode 100644 index 0000000000..fb26d44d30 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/DruidDataSource.java @@ -0,0 +1,150 @@ +/* + * 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; + + +import org.apache.eventmesh.admin.server.web.utils.EncryptUtil; +import org.apache.eventmesh.admin.server.web.utils.ParamType; + +import org.apache.commons.lang3.StringUtils; + +import java.io.IOException; +import java.sql.SQLException; + +import javax.sql.DataSource; + +import org.springframework.beans.factory.annotation.Value; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.ComponentScan; +import org.springframework.context.annotation.Configuration; +import org.springframework.context.annotation.Primary; + +import lombok.extern.slf4j.Slf4j; + +@Configuration +@ComponentScan +@Slf4j +public class DruidDataSource { + + @Value("${spring.datasource.url}") + private String dbUrl; + + @Value("${spring.datasource.username}") + private String username; + + @Value("${spring.datasource.password}") + private String password; + + @Value("${spring.datasource.driver-class-name}") + private String driverClassName; + + @Value("${spring.datasource.initialSize}") + private int initialSize; + + @Value("${spring.datasource.minIdle}") + private int minIdle; + + @Value("${spring.datasource.maxActive}") + private int maxActive; + + @Value("${spring.datasource.maxWait}") + private int maxWait; + + @Value("${spring.datasource.timeBetweenEvictionRunsMillis}") + private int timeBetweenEvictionRunsMillis; + + @Value("${spring.datasource.minEvictableIdleTimeMillis}") + private int minEvictableIdleTimeMillis; + + @Value("${spring.datasource.validationQuery}") + private String validationQuery; + + @Value("${spring.datasource.testWhileIdle}") + private boolean testWhileIdle; + + @Value("${spring.datasource.testOnBorrow}") + private boolean testOnBorrow; + + @Value("${spring.datasource.testOnReturn}") + private boolean testOnReturn; + + @Value("${spring.datasource.poolPreparedStatements}") + private boolean poolPreparedStatements; + + @Value("${spring.datasource.maxPoolPreparedStatementPerConnectionSize}") + private int maxPoolPreparedStatementPerConnectionSize; + + @Value("${spring.datasource.filters}") + private String filters; + + @Value("{spring.datasource.connectionProperties}") + private String connectionProperties; + + @Value("${sysPubKey}") + private String sysPubKeyStr; + + @Value("${appPrivKey}") + private String appPrivKeyStr; + + + @Bean + @Primary + public DataSource dataSource() throws Exception { + try (com.alibaba.druid.pool.DruidDataSource datasource = new com.alibaba.druid.pool.DruidDataSource()) { + datasource.setUrl(this.dbUrl); + datasource.setUsername(username); + datasource.setPassword(rsaDecrypt(sysPubKeyStr, appPrivKeyStr, password)); + datasource.setDriverClassName(driverClassName); + datasource.setInitialSize(initialSize); + datasource.setMinIdle(minIdle); + datasource.setMaxActive(maxActive); + datasource.setMaxWait(maxWait); + datasource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis); + datasource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis); + datasource.setValidationQuery(validationQuery); + datasource.setTestWhileIdle(testWhileIdle); + datasource.setTestOnBorrow(testOnBorrow); + datasource.setTestOnReturn(testOnReturn); + datasource.setPoolPreparedStatements(poolPreparedStatements); + datasource.setMaxPoolPreparedStatementPerConnectionSize(maxPoolPreparedStatementPerConnectionSize); + try { + datasource.setFilters(filters); + } catch (SQLException e) { + log.error("druid configuration initialization filter", e); + } + datasource.setConnectionProperties(connectionProperties); + + return datasource; + } + } + + public static String rsaDecrypt(String sysPubKeyStr, String appPrivKeyStr, String encrtyptText) throws IOException { + if (StringUtils.isNotBlank(encrtyptText) && encrtyptText.length() > "{RSA}".length() && encrtyptText.startsWith("{RSA}")) { + String text = encrtyptText.startsWith("{RSA}") ? encrtyptText.substring("{RSA}".length()) : encrtyptText; + + try { + return EncryptUtil.decrypt(ParamType.STRING, sysPubKeyStr, ParamType.STRING, appPrivKeyStr, ParamType.STRING, text); + } catch (Exception e) { + throw new RuntimeException("decrypt error", e); + } + } else { + return encrtyptText; + } + } + +} diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshMonitor.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshMonitor.java new file mode 100644 index 0000000000..0507464b5b --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshMonitor.java @@ -0,0 +1,52 @@ +/* + * 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.entity; + +import java.io.Serializable; +import java.util.Date; + +import com.baomidou.mybatisplus.annotation.IdType; +import com.baomidou.mybatisplus.annotation.TableId; +import com.baomidou.mybatisplus.annotation.TableName; + +import lombok.Data; + +/** + * TableName event_mesh_monitor + */ +@TableName(value = "event_mesh_monitor") +@Data +public class EventMeshMonitor implements Serializable { + + @TableId(type = IdType.AUTO) + private Integer id; + + private String taskID; + private String jobID; + private String address; + private String transportType; + private String connectorStage; + private Long totalReqNum; + private Long totalTimeCost; + private Long maxTimeCost; + private Long avgTimeCost; + private Double tps; + private Date createTime; + + private static final long serialVersionUID = 1L; +} \ No newline at end of file diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshWeredisPosition.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshWeredisPosition.java new file mode 100644 index 0000000000..2117230826 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/entity/EventMeshWeredisPosition.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.admin.server.web.db.entity; + +import java.io.Serializable; +import java.util.Date; + +import com.baomidou.mybatisplus.annotation.IdType; +import com.baomidou.mybatisplus.annotation.TableId; +import com.baomidou.mybatisplus.annotation.TableName; + +import lombok.Data; + +/** + * TableName event_mesh_weredis_position + */ +@TableName(value = "event_mesh_weredis_position") +@Data +public class EventMeshWeredisPosition implements Serializable { + @TableId(type = IdType.AUTO) + private Integer id; + + private String jobID; + + // connection run address + private String address; + + private String clusterName; + + private String partitionName; + + private String masterReplid; + + //weredis run host + private String host; + + private Long replOffset = -1L; + + private Date createTime; + + private Date updateTime; + + private static final long serialVersionUID = 1L; +} \ No newline at end of file diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/mapper/EventMeshMonitorMapper.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/mapper/EventMeshMonitorMapper.java new file mode 100644 index 0000000000..db77224637 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/mapper/EventMeshMonitorMapper.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.admin.server.web.db.mapper; + +import org.apache.eventmesh.admin.server.web.db.entity.EventMeshMonitor; + +import org.apache.ibatis.annotations.Mapper; + +import com.baomidou.mybatisplus.core.mapper.BaseMapper; + +/** + * event_mesh_monitor + * Entity org.apache.eventmesh.admin.server.web.db.entity.EventMeshMonitor + */ +@Mapper +public interface EventMeshMonitorMapper extends BaseMapper { + +} + + + + diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/EventMeshMonitorService.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/EventMeshMonitorService.java new file mode 100644 index 0000000000..4180f82a97 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/EventMeshMonitorService.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.admin.server.web.db.service; + +import org.apache.eventmesh.admin.server.web.db.entity.EventMeshMonitor; + +import com.baomidou.mybatisplus.extension.service.IService; + +/** + * event_mesh_monitor + */ +public interface EventMeshMonitorService extends IService { + +} diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/EventMeshTaskInfoService.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/EventMeshTaskInfoService.java index dc35cfe071..04da6a7952 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/EventMeshTaskInfoService.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/EventMeshTaskInfoService.java @@ -18,6 +18,10 @@ package org.apache.eventmesh.admin.server.web.db.service; import org.apache.eventmesh.admin.server.web.db.entity.EventMeshTaskInfo; +import org.apache.eventmesh.common.remote.request.QueryTaskInfoRequest; +import org.apache.eventmesh.common.remote.response.QueryTaskInfoResponse; + +import java.util.List; import com.baomidou.mybatisplus.extension.service.IService; @@ -26,4 +30,8 @@ */ public interface EventMeshTaskInfoService extends IService { + List queryTaskInfo(QueryTaskInfoRequest taskInfoRequest); + + // boolean deleteTaskByTaskID(String taskID); + } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshMonitorServiceImpl.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshMonitorServiceImpl.java new file mode 100644 index 0000000000..ebb4220000 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshMonitorServiceImpl.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.EventMeshMonitor; +import org.apache.eventmesh.admin.server.web.db.mapper.EventMeshMonitorMapper; +import org.apache.eventmesh.admin.server.web.db.service.EventMeshMonitorService; + +import org.springframework.stereotype.Service; + +import com.baomidou.mybatisplus.extension.service.impl.ServiceImpl; + +/** + * event_mesh_monitor + */ +@Service +public class EventMeshMonitorServiceImpl extends ServiceImpl + implements EventMeshMonitorService { + +} + + + + diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshTaskInfoServiceImpl.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshTaskInfoServiceImpl.java index 9568b63671..91acb51a76 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshTaskInfoServiceImpl.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/db/service/impl/EventMeshTaskInfoServiceImpl.java @@ -17,23 +17,307 @@ package org.apache.eventmesh.admin.server.web.db.service.impl; +import org.apache.eventmesh.admin.server.web.db.entity.EventMeshDataSource; +import org.apache.eventmesh.admin.server.web.db.entity.EventMeshJobInfo; +import org.apache.eventmesh.admin.server.web.db.entity.EventMeshMysqlPosition; import org.apache.eventmesh.admin.server.web.db.entity.EventMeshTaskInfo; import org.apache.eventmesh.admin.server.web.db.mapper.EventMeshTaskInfoMapper; +import org.apache.eventmesh.admin.server.web.db.service.EventMeshDataSourceService; +import org.apache.eventmesh.admin.server.web.db.service.EventMeshJobInfoService; +import org.apache.eventmesh.admin.server.web.db.service.EventMeshMysqlPositionService; import org.apache.eventmesh.admin.server.web.db.service.EventMeshTaskInfoService; +import org.apache.eventmesh.common.remote.JobState; +import org.apache.eventmesh.common.remote.TaskState; +import org.apache.eventmesh.common.remote.request.QueryTaskInfoRequest; +import org.apache.eventmesh.common.remote.response.QueryTaskInfoResponse; +import org.apache.commons.lang3.ObjectUtils; +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.List; + +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Service; +import org.springframework.transaction.annotation.Transactional; +import com.baomidou.mybatisplus.core.toolkit.Wrappers; +import com.baomidou.mybatisplus.extension.plugins.pagination.Page; import com.baomidou.mybatisplus.extension.service.impl.ServiceImpl; +import lombok.extern.slf4j.Slf4j; + /** * event_mesh_task_info */ +@Slf4j @Service public class EventMeshTaskInfoServiceImpl extends ServiceImpl - implements EventMeshTaskInfoService { + implements EventMeshTaskInfoService { + + @Autowired + private EventMeshTaskInfoMapper taskInfoMapper; + + @Autowired + private EventMeshJobInfoService jobInfoService; + + @Autowired + private EventMeshDataSourceService dataSourceService; + + @Autowired + private EventMeshMysqlPositionService mysqlPositionService; + + @Override + public List queryTaskInfo(QueryTaskInfoRequest taskInfoRequest) { + + log.info("receive query task info request:{}", taskInfoRequest); + + List queryTaskInfoResponseList = new ArrayList<>(); + + Integer currentPage = taskInfoRequest.getCurrentPage(); + Integer pageSize = taskInfoRequest.getPageSize(); + + // query by page + if (StringUtils.isEmpty(taskInfoRequest.getTaskID()) + && currentPage != null + && pageSize != null) { + + Page page = new Page<>(); + page.setCurrent(currentPage); + page.setSize(pageSize); + List eventMeshTaskInfoList = taskInfoMapper.selectPage(page, Wrappers.query() + .ne("taskState", TaskState.DELETE.name())).getRecords(); + queryTaskInfoResponseList = getQueryTaskInfoResponses(eventMeshTaskInfoList, queryTaskInfoResponseList); + + } + + if (StringUtils.isNotEmpty(taskInfoRequest.getTaskID()) || StringUtils.isNotEmpty(taskInfoRequest.getTaskID())) { + queryTaskInfoResponseList = eventMeshTaskInfoList(taskInfoRequest); + } + + // if (StringUtils.isNotEmpty(taskInfoRequest.getJobType())) { + // + // } + // + // if (StringUtils.isNotEmpty(taskInfoRequest.getSourceDataID())) { + // + // } + // + // if (StringUtils.isNotEmpty(taskInfoRequest.getTargetDataID())) { + // + // } + // + // if (StringUtils.isNotEmpty(taskInfoRequest.getIp())) { + // + // } + // + // if (StringUtils.isNotEmpty(taskInfoRequest.getSourceTableName())) { + // + // } + // + // if (StringUtils.isNotEmpty(taskInfoRequest.getTaskMathID())) { + // + // } + + log.info("query event mesh task info response result:{}", queryTaskInfoResponseList); + + return queryTaskInfoResponseList; + } + + @Transactional + private List eventMeshTaskInfoList(QueryTaskInfoRequest taskInfoRequest) { + + List eventMeshTaskInfoList = new ArrayList<>(); + + Page page = new Page<>(); + page.setCurrent(taskInfoRequest.getCurrentPage()); + page.setSize(taskInfoRequest.getPageSize()); + + if (StringUtils.isNotEmpty(taskInfoRequest.getTaskID())) { + eventMeshTaskInfoList = taskInfoMapper.selectPage(page, Wrappers.query() + .eq("taskID", taskInfoRequest.getTaskID()) + .ne("taskState", TaskState.DELETE.name())) + .getRecords(); + } + + if (StringUtils.isNotEmpty(taskInfoRequest.getTaskDesc())) { + eventMeshTaskInfoList = taskInfoMapper.selectPage(page, Wrappers.query() + .like("taskDesc", taskInfoRequest.getTaskDesc()) + .ne("jobState", JobState.DELETE.name())) + .getRecords(); + } + + List eventMeshTaskInfos = new ArrayList<>(); + + List queryTaskInfoResponse = getQueryTaskInfoResponses(eventMeshTaskInfoList, eventMeshTaskInfos); + log.info("query task info result queryTaskInfoResponse:{}", queryTaskInfoResponse); + + return queryTaskInfoResponse; + } + + private List getQueryTaskInfoResponses(List eventMeshTaskInfoList, + List eventMeshTaskInfos) { + + for (EventMeshTaskInfo meshTaskInfo : eventMeshTaskInfoList) { + QueryTaskInfoResponse eventMeshTaskInfo = initEventMeshTaskInfo(meshTaskInfo); + eventMeshTaskInfos.add(eventMeshTaskInfo); + } + + if (!eventMeshTaskInfoList.isEmpty()) { + List eventMeshJobInfoList = new ArrayList<>(); + for (QueryTaskInfoResponse eventMeshTaskInfo : eventMeshTaskInfos) { + List eventMeshJobInfos = jobInfoService.list(Wrappers.query() + .eq("taskID", eventMeshTaskInfo.getTaskID()) + .ne("jobState", JobState.DELETE.name())); + + for (EventMeshJobInfo eventMeshJobInfo : eventMeshJobInfos) { + QueryTaskInfoResponse.EventMeshJobInfo eventMeshJobInfoCovert = initEventMeshJobInfo(eventMeshJobInfo); + eventMeshJobInfoList.add(eventMeshJobInfoCovert); + } + + if (!eventMeshJobInfoList.isEmpty()) { + for (QueryTaskInfoResponse.EventMeshJobInfo eventMeshJobInfo : eventMeshJobInfoList) { + QueryTaskInfoResponse.EventMeshDataSource dataSource = covertEventMeshDataSource( + querySourceOrSinkData(eventMeshJobInfo.getSourceData())); + QueryTaskInfoResponse.EventMeshDataSource dataSink = covertEventMeshDataSource( + querySourceOrSinkData(eventMeshJobInfo.getTargetData())); + + EventMeshMysqlPosition eventMeshMysqlPosition = mysqlPositionService.getOne(Wrappers.query().eq( + "jobID", + eventMeshJobInfo.getJobID() + )); + + + QueryTaskInfoResponse.EventMeshMysqlPosition mysqlPosition = covertEventMeshMysqlPosition(eventMeshMysqlPosition); + + eventMeshJobInfo.setEventMeshMysqlPosition(mysqlPosition); + eventMeshJobInfo.setDataSource(dataSource); + eventMeshJobInfo.setDataSink(dataSink); + } + } + + // set job info to same taskID + eventMeshTaskInfo.setEventMeshJobInfoList(eventMeshJobInfoList); + } + } + + List queryTaskInfoResponse = new ArrayList<>(); + if (!eventMeshTaskInfos.isEmpty()) { + queryTaskInfoResponse.addAll(eventMeshTaskInfos); + } + + return queryTaskInfoResponse; + } + + /** + * QueryTaskInfoResponse.EventMeshDataSource covert + * + * @param eventMeshData EventMeshDataSource + * @return meshData + */ + private static QueryTaskInfoResponse.EventMeshDataSource covertEventMeshDataSource(EventMeshDataSource eventMeshData) { + QueryTaskInfoResponse.EventMeshDataSource meshData = new QueryTaskInfoResponse.EventMeshDataSource(); + if (ObjectUtils.isEmpty(eventMeshData)) { + return null; + } + meshData.setId(eventMeshData.getId()); + meshData.setDataType(eventMeshData.getDataType()); + meshData.setConfiguration(eventMeshData.getConfiguration()); + meshData.setConfigurationClass(eventMeshData.getConfigurationClass()); + meshData.setDescription(eventMeshData.getDescription()); + meshData.setRegion(eventMeshData.getRegion()); + meshData.setCreateUid(eventMeshData.getCreateUid()); + meshData.setUpdateUid(eventMeshData.getUpdateUid()); + meshData.setCreateTime(eventMeshData.getCreateTime()); + meshData.setUpdateTime(eventMeshData.getUpdateTime()); + return meshData; + } -} + /** + * getSourceOrSinkData + * + * @param id id + * @return EventMeshDataSource + */ + private EventMeshDataSource querySourceOrSinkData(Integer id) { + return dataSourceService.getOne(Wrappers.query().eq( + "id", + id)); + } + /** + * QueryTaskInfoResponse.EventMeshMysqlPosition + * + * @param mysqlPosition EventMeshMysqlPosition + * @return position + */ + private static QueryTaskInfoResponse.EventMeshMysqlPosition covertEventMeshMysqlPosition(EventMeshMysqlPosition mysqlPosition) { + QueryTaskInfoResponse.EventMeshMysqlPosition position = new QueryTaskInfoResponse.EventMeshMysqlPosition(); + if (ObjectUtils.isEmpty(mysqlPosition)) { + return null; + } + position.setId(mysqlPosition.getId()); + position.setJobID(mysqlPosition.getJobID()); + position.setServerUUID(mysqlPosition.getServerUUID()); + position.setAddress(mysqlPosition.getAddress()); + position.setPosition(mysqlPosition.getPosition()); + position.setGtid(mysqlPosition.getGtid()); + position.setCurrentGtid(mysqlPosition.getCurrentGtid()); + position.setTimestamp(mysqlPosition.getTimestamp()); + position.setJournalName(mysqlPosition.getJournalName()); + position.setCreateTime(mysqlPosition.getCreateTime()); + position.setUpdateTime(mysqlPosition.getUpdateTime()); + return position; + } + /** + * EventMeshJobInfo covert + * + * @param eventMeshJobInfo EventMeshJobInfo + * @return QueryTaskInfoResponse.EventMeshJobInfo + */ + private static QueryTaskInfoResponse.EventMeshJobInfo initEventMeshJobInfo(EventMeshJobInfo eventMeshJobInfo) { + QueryTaskInfoResponse.EventMeshJobInfo eventMeshJobInfoCovert = new QueryTaskInfoResponse.EventMeshJobInfo(); + if (ObjectUtils.isEmpty(eventMeshJobInfo)) { + return null; + } + eventMeshJobInfoCovert.setId(eventMeshJobInfo.getId()); + eventMeshJobInfoCovert.setJobID(eventMeshJobInfo.getJobID()); + eventMeshJobInfoCovert.setJobDesc(eventMeshJobInfo.getJobDesc()); + eventMeshJobInfoCovert.setTaskID(eventMeshJobInfo.getTaskID()); + eventMeshJobInfoCovert.setTransportType(eventMeshJobInfo.getTransportType()); + eventMeshJobInfoCovert.setSourceData(eventMeshJobInfo.getSourceData()); + eventMeshJobInfoCovert.setTargetData(eventMeshJobInfo.getTargetData()); + eventMeshJobInfoCovert.setJobState(eventMeshJobInfo.getJobState()); + eventMeshJobInfoCovert.setJobType(eventMeshJobInfo.getJobType()); + eventMeshJobInfoCovert.setFromRegion(eventMeshJobInfo.getFromRegion()); + eventMeshJobInfoCovert.setRunningRegion(eventMeshJobInfo.getRunningRegion()); + eventMeshJobInfoCovert.setCreateUid(eventMeshJobInfo.getCreateUid()); + eventMeshJobInfoCovert.setUpdateUid(eventMeshJobInfo.getUpdateUid()); + eventMeshJobInfoCovert.setCreateTime(eventMeshJobInfo.getCreateTime()); + eventMeshJobInfoCovert.setUpdateTime(eventMeshJobInfo.getUpdateTime()); + return eventMeshJobInfoCovert; + } + /** + * EventMeshTaskInfo covert + * + * @param meshTaskInfo EventMeshTaskInfo + * @return QueryTaskInfoResponse + */ + private static QueryTaskInfoResponse initEventMeshTaskInfo(EventMeshTaskInfo meshTaskInfo) { + QueryTaskInfoResponse eventMeshTaskInfo = new QueryTaskInfoResponse(); + eventMeshTaskInfo.setId(meshTaskInfo.getId()); + eventMeshTaskInfo.setTaskID(meshTaskInfo.getTaskID()); + eventMeshTaskInfo.setTaskDesc(meshTaskInfo.getTaskDesc()); + eventMeshTaskInfo.setTaskState(meshTaskInfo.getTaskState()); + eventMeshTaskInfo.setSourceRegion(meshTaskInfo.getSourceRegion()); + eventMeshTaskInfo.setTargetRegion(meshTaskInfo.getTargetRegion()); + eventMeshTaskInfo.setCreateUid(meshTaskInfo.getCreateUid()); + eventMeshTaskInfo.setUpdateUid(meshTaskInfo.getUpdateUid()); + eventMeshTaskInfo.setCreateTime(meshTaskInfo.getCreateTime()); + eventMeshTaskInfo.setUpdateTime(meshTaskInfo.getUpdateTime()); + return eventMeshTaskInfo; + } +} \ No newline at end of file diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportJobRequestHandler.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportJobRequestHandler.java index ea836ce7aa..c876014f63 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportJobRequestHandler.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportJobRequestHandler.java @@ -19,19 +19,29 @@ import org.apache.eventmesh.admin.server.web.db.entity.EventMeshJobInfo; import org.apache.eventmesh.admin.server.web.handler.BaseRequestHandler; +import org.apache.eventmesh.admin.server.web.pojo.TaskDetail; import org.apache.eventmesh.admin.server.web.service.job.JobInfoBizService; +import org.apache.eventmesh.admin.server.web.service.position.PositionBizService; import org.apache.eventmesh.common.protocol.grpc.adminserver.Metadata; +import org.apache.eventmesh.common.remote.JobState; +import org.apache.eventmesh.common.remote.TransportType; +import org.apache.eventmesh.common.remote.datasource.DataSourceType; import org.apache.eventmesh.common.remote.exception.ErrorCode; +import org.apache.eventmesh.common.remote.offset.RecordPosition; +import org.apache.eventmesh.common.remote.request.RecordPositionRequest; import org.apache.eventmesh.common.remote.request.ReportJobRequest; import org.apache.eventmesh.common.remote.response.SimpleResponse; import org.apache.commons.lang3.StringUtils; +import java.util.List; + import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; import lombok.extern.slf4j.Slf4j; + @Component @Slf4j public class ReportJobRequestHandler extends BaseRequestHandler { @@ -39,6 +49,9 @@ public class ReportJobRequestHandler extends BaseRequestHandler recordPositionList = + positionBizService.getPositionByJobID(taskDetail.getIncreaseTask().getJobID(), DataSourceType.MYSQL); + if (!recordPositionList.isEmpty()) { + log.info("skip record position because of increase job has exist position.jobID:{},position list size:{}", jobInfo.getJobID(), + recordPositionList.size()); + return true; + } + + RecordPositionRequest recordPositionRequest = new RecordPositionRequest(); + recordPositionRequest.setFullJobID(taskDetail.getFullTask().getJobID()); + recordPositionRequest.setIncreaseJobID(taskDetail.getIncreaseTask().getJobID()); + recordPositionRequest.setUpdateState(request.getState()); + recordPositionRequest.setAddress(request.getAddress()); + TransportType currentTransportType = TransportType.getTransportType(jobInfo.getTransportType()); + recordPositionRequest.setDataSourceType(currentTransportType.getSrc()); + return positionBizService.recordPosition(recordPositionRequest, metadata); + } + } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportMonitorHandler.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportMonitorHandler.java new file mode 100644 index 0000000000..a36939bb88 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportMonitorHandler.java @@ -0,0 +1,97 @@ +/* + * 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.handler.impl; + +import org.apache.eventmesh.admin.server.AdminServerProperties; +import org.apache.eventmesh.admin.server.web.db.entity.EventMeshJobInfo; +import org.apache.eventmesh.admin.server.web.handler.BaseRequestHandler; +import org.apache.eventmesh.admin.server.web.service.job.JobInfoBizService; +import org.apache.eventmesh.admin.server.web.service.monitor.MonitorBizService; +import org.apache.eventmesh.common.protocol.grpc.adminserver.Metadata; +import org.apache.eventmesh.common.remote.exception.ErrorCode; +import org.apache.eventmesh.common.remote.request.ReportMonitorRequest; +import org.apache.eventmesh.common.remote.response.SimpleResponse; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.http.ResponseEntity; +import org.springframework.stereotype.Component; +import org.springframework.web.client.RestTemplate; + +import lombok.extern.slf4j.Slf4j; + +@Component +@Slf4j +public class ReportMonitorHandler extends BaseRequestHandler { + + @Autowired + private MonitorBizService monitorService; + + @Autowired + JobInfoBizService jobInfoBizService; + + @Autowired + private AdminServerProperties properties; + + @Override + protected SimpleResponse handler(ReportMonitorRequest request, Metadata metadata) { + if (StringUtils.isAnyBlank(request.getTaskID(), request.getJobID(), request.getAddress())) { + log.info("report monitor request [{}] illegal", request); + return SimpleResponse.fail(ErrorCode.BAD_REQUEST, "request task id,job id or address is none"); + } + + String jobID = request.getJobID(); + EventMeshJobInfo jobInfo = jobInfoBizService.getJobInfo(jobID); + if (jobInfo == null || StringUtils.isBlank(jobInfo.getFromRegion())) { + log.info("report monitor job info [{}] illegal", request); + return SimpleResponse.fail(ErrorCode.BAD_REQUEST, "job info is null or fromRegion is blank,job id:" + jobID); + } + String fromRegion = jobInfo.getFromRegion(); + String transportType = jobInfo.getTransportType(); + if (StringUtils.isEmpty(request.getTransportType())) { + request.setTransportType(transportType); + } + String localRegion = properties.getRegion(); + log.info("report monitor request from region:{},localRegion:{},request:{}", fromRegion, localRegion, request); + if (fromRegion.equalsIgnoreCase(localRegion)) { + return monitorService.reportMonitorRecord(request) ? SimpleResponse.success() : + SimpleResponse.fail(ErrorCode.INTERNAL_ERR, "save monitor " + + "request fail"); + } else { + List adminServerList = Arrays.asList(properties.getAdminServerList().get(fromRegion).split(";")); + if (adminServerList == null || adminServerList.isEmpty()) { + throw new RuntimeException("No admin server available for region: " + fromRegion); + } + String targetUrl = adminServerList.get(new Random().nextInt(adminServerList.size())) + "/eventmesh/admin/reportMonitor"; + log.info("start transfer monitor request to from region admin server. from region:{}, targetUrl:{}", fromRegion, targetUrl); + RestTemplate restTemplate = new RestTemplate(); + ResponseEntity response = restTemplate.postForEntity(targetUrl, request, String.class); + if (!response.getStatusCode().is2xxSuccessful()) { + log.error("transfer monitor request to from region admin server error. from region:{}, targetUrl:{}", fromRegion, targetUrl); + return SimpleResponse.fail(ErrorCode.INTERNAL_ERR, + "save monitor request fail,code:" + response.getStatusCode() + ",msg:" + response.getBody()); + } + return SimpleResponse.success(); + } + } +} diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportVerifyHandler.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportVerifyHandler.java index 9844f47c6a..e7f1d1257f 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportVerifyHandler.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/handler/impl/ReportVerifyHandler.java @@ -29,6 +29,7 @@ import org.apache.commons.lang3.StringUtils; +import java.util.Arrays; import java.util.List; import java.util.Random; @@ -75,7 +76,7 @@ protected SimpleResponse handler(ReportVerifyRequest request, Metadata metadata) + "request fail"); } else { log.info("start transfer report verify to from region admin server. from region:{}", fromRegion); - List adminServerList = properties.getAdminServerList().get(fromRegion); + List adminServerList = Arrays.asList(properties.getAdminServerList().get(fromRegion).split(";")); if (adminServerList == null || adminServerList.isEmpty()) { throw new RuntimeException("No admin server available for region: " + fromRegion); } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/BinlogPosition.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/BinlogPosition.java new file mode 100644 index 0000000000..5bd8daab10 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/BinlogPosition.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.admin.server.web.pojo; + + +import lombok.Data; + +@Data +public class BinlogPosition { + private String file; + private Long position; +} diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/TaskDetail.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/TaskDetail.java index 86f5342f35..2b174209e2 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/TaskDetail.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/pojo/TaskDetail.java @@ -17,8 +17,18 @@ package org.apache.eventmesh.admin.server.web.pojo; +import org.apache.eventmesh.admin.server.web.db.entity.EventMeshJobInfo; + +import lombok.Data; + /** * Description: */ +@Data public class TaskDetail { + + private EventMeshJobInfo fullTask; + + private EventMeshJobInfo increaseTask; + } 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 76df629e69..c200d9801a 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 @@ -28,6 +28,7 @@ import org.apache.eventmesh.admin.server.web.db.service.EventMeshJobInfoService; import org.apache.eventmesh.admin.server.web.db.service.EventMeshRuntimeHeartbeatService; import org.apache.eventmesh.admin.server.web.pojo.JobDetail; +import org.apache.eventmesh.admin.server.web.pojo.TaskDetail; import org.apache.eventmesh.admin.server.web.service.datasource.DataSourceBizService; import org.apache.eventmesh.admin.server.web.service.position.PositionBizService; import org.apache.eventmesh.common.config.connector.Config; @@ -171,7 +172,7 @@ public List createJobs(List jobs) { entityList.add(entity); } int changed = jobInfoExtService.batchSave(entityList); - if (changed != jobs.size()) { + if (changed != entityList.size()) { throw new AdminServerRuntimeException(ErrorCode.INTERNAL_ERR, String.format("create [%d] jobs of not match expect [%d]", changed, jobs.size())); } @@ -241,8 +242,14 @@ public EventMeshJobInfo getJobInfo(String jobID) { if (jobID == null) { return null; } - EventMeshJobInfo job = jobInfoService.getOne(Wrappers.query().eq("jobID", jobID)); - return job; + return jobInfoService.getOne(Wrappers.query().eq("jobID", jobID)); + } + + public List getJobsByTaskID(String taskID) { + if (taskID == null) { + return null; + } + return jobInfoService.list(Wrappers.query().eq("taskID", taskID)); } public void checkJobInfo() { @@ -253,19 +260,41 @@ public void checkJobInfo() { if (StringUtils.isEmpty(jobID)) { continue; } - EventMeshRuntimeHeartbeat heartbeat = heartbeatService.getOne(Wrappers.query().eq("jobID", jobID)); - if (heartbeat == null) { + List heartbeatList = heartbeatService.list((Wrappers.query().eq("jobID", jobID))); + if (heartbeatList == null || heartbeatList.size() == 0) { continue; } // if last heart beat update time have delay three period.print job heart beat delay warn long currentTimeStamp = System.currentTimeMillis(); - if (currentTimeStamp - heartbeat.getUpdateTime().getTime() > 3 * heatBeatPeriod) { + if (currentTimeStamp - heartbeatList.get(0).getUpdateTime().getTime() > 3 * heatBeatPeriod) { log.warn("current job heart heart has delay.jobID:{},currentTimeStamp:{},last update time:{}", jobID, currentTimeStamp, - heartbeat.getUpdateTime()); + heartbeatList.get(0).getUpdateTime()); } } } + public TaskDetail getTaskDetail(String taskID, DataSourceType dataSourceType) { + TaskDetail taskDetail = new TaskDetail(); + List jobInfoList = getJobsByTaskID(taskID); + if (jobInfoList == null || jobInfoList.size() == 0) { + return taskDetail; + } + for (EventMeshJobInfo jobInfo : jobInfoList) { + TransportType currentTransportType = TransportType.getTransportType(jobInfo.getTransportType()); + JobType jobType = JobType.fromIndex(jobInfo.getJobType()); + if (currentTransportType.getSrc().equals(dataSourceType)) { + if (jobType.name().equalsIgnoreCase(JobType.FULL.name())) { + taskDetail.setFullTask(jobInfo); + } + if (jobType.name().equalsIgnoreCase(JobType.INCREASE.name())) { + taskDetail.setIncreaseTask(jobInfo); + } + } + } + return taskDetail; + } + + } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/monitor/MonitorBizService.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/monitor/MonitorBizService.java new file mode 100644 index 0000000000..3377334144 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/monitor/MonitorBizService.java @@ -0,0 +1,111 @@ +/* + * 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.service.monitor; + +import org.apache.eventmesh.admin.server.web.db.entity.EventMeshMonitor; +import org.apache.eventmesh.admin.server.web.db.service.EventMeshMonitorService; +import org.apache.eventmesh.common.remote.request.QueryTaskMonitorRequest; +import org.apache.eventmesh.common.remote.request.ReportMonitorRequest; +import org.apache.eventmesh.common.remote.response.QueryTaskMonitorResponse; +import org.apache.eventmesh.common.remote.task.TaskMonitor; +import org.apache.eventmesh.common.utils.JsonUtils; + +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.List; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Service; + +import com.baomidou.mybatisplus.core.conditions.query.QueryWrapper; +import com.baomidou.mybatisplus.core.metadata.OrderItem; +import com.baomidou.mybatisplus.extension.plugins.pagination.Page; + +import lombok.extern.slf4j.Slf4j; + +@Service +@Slf4j +public class MonitorBizService { + + @Autowired + private EventMeshMonitorService monitorService; + + public boolean reportMonitorRecord(ReportMonitorRequest request) { + EventMeshMonitor monitor = new EventMeshMonitor(); + monitor.setTaskID(request.getTaskID()); + monitor.setJobID(request.getJobID()); + monitor.setAddress(request.getAddress()); + monitor.setTransportType(request.getTransportType()); + monitor.setConnectorStage(request.getConnectorStage()); + monitor.setTotalReqNum(request.getTotalReqNum()); + monitor.setTotalTimeCost(request.getTotalTimeCost()); + monitor.setMaxTimeCost(request.getMaxTimeCost()); + monitor.setAvgTimeCost(request.getAvgTimeCost()); + monitor.setTps(request.getTps()); + return monitorService.save(monitor); + } + + public QueryTaskMonitorResponse queryTaskMonitors(QueryTaskMonitorRequest request) { + if (StringUtils.isBlank(request.getTaskID())) { + throw new RuntimeException("task id is empty"); + } + long limit = request.getLimit(); + if (limit <= 0) { + log.info("query task monitor limit:{},use default value:{}", limit, 10); + limit = 10; + } + + Page queryPage = new Page<>(); + queryPage.setCurrent(1); + queryPage.setSize(limit); + queryPage.addOrder(OrderItem.desc("createTime")); + + QueryWrapper queryWrapper = new QueryWrapper(); + queryWrapper.eq("taskID", request.getTaskID()); + if (StringUtils.isNotEmpty(request.getJobID())) { + queryWrapper.eq("jobID", request.getJobID()); + } + List eventMeshMonitors = monitorService.list(queryPage, queryWrapper); + List taskMonitorList = new ArrayList<>(); + if (eventMeshMonitors != null) { + log.info("query event mesh monitor size:{}", eventMeshMonitors.size()); + if (log.isDebugEnabled()) { + log.debug("query event mesh monitor content:{}", JsonUtils.toJSONString(eventMeshMonitors)); + } + for (EventMeshMonitor eventMeshMonitor : eventMeshMonitors) { + TaskMonitor monitor = new TaskMonitor(); + monitor.setTaskID(eventMeshMonitor.getTaskID()); + monitor.setJobID(eventMeshMonitor.getJobID()); + monitor.setAddress(eventMeshMonitor.getAddress()); + monitor.setTransportType(eventMeshMonitor.getTransportType()); + monitor.setConnectorStage(eventMeshMonitor.getConnectorStage()); + monitor.setTotalReqNum(eventMeshMonitor.getTotalReqNum()); + monitor.setTotalTimeCost(eventMeshMonitor.getTotalTimeCost()); + monitor.setMaxTimeCost(eventMeshMonitor.getMaxTimeCost()); + monitor.setAvgTimeCost(eventMeshMonitor.getAvgTimeCost()); + monitor.setTps(eventMeshMonitor.getTps()); + monitor.setCreateTime(eventMeshMonitor.getCreateTime()); + taskMonitorList.add(monitor); + } + } + QueryTaskMonitorResponse queryTaskMonitorResponse = new QueryTaskMonitorResponse(); + queryTaskMonitorResponse.setTaskMonitors(taskMonitorList); + return queryTaskMonitorResponse; + } +} diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/IRecordPositionHandler.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/IRecordPositionHandler.java new file mode 100644 index 0000000000..fa38e14320 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/IRecordPositionHandler.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.admin.server.web.service.position; + +import org.apache.eventmesh.common.protocol.grpc.adminserver.Metadata; +import org.apache.eventmesh.common.remote.request.RecordPositionRequest; + +/** + * IRecordPositionHandler + */ +public interface IRecordPositionHandler { + + boolean handler(RecordPositionRequest request, Metadata metadata); + +} diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/PositionBizService.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/PositionBizService.java index c40fc9e7e5..0c4cd7a423 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/PositionBizService.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/PositionBizService.java @@ -23,6 +23,7 @@ import org.apache.eventmesh.common.remote.exception.ErrorCode; import org.apache.eventmesh.common.remote.offset.RecordPosition; import org.apache.eventmesh.common.remote.request.FetchPositionRequest; +import org.apache.eventmesh.common.remote.request.RecordPositionRequest; import org.apache.eventmesh.common.remote.request.ReportPositionRequest; import java.util.List; @@ -80,4 +81,10 @@ public List getPositionByJobID(String jobID, DataSourceType type request.setJobID(jobID); return handler.handler(request, null); } + + public boolean recordPosition(RecordPositionRequest request, Metadata metadata) { + isValidatePositionRequest(request.getDataSourceType()); + IRecordPositionHandler handler = factory.getHandler(request.getDataSourceType()); + return handler.handler(request, metadata); + } } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/PositionHandler.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/PositionHandler.java index e09c1a3837..9cbaf3fad6 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/PositionHandler.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/PositionHandler.java @@ -19,7 +19,7 @@ import org.apache.eventmesh.common.remote.datasource.DataSourceType; -public abstract class PositionHandler implements IReportPositionHandler, IFetchPositionHandler { +public abstract class PositionHandler implements IReportPositionHandler, IFetchPositionHandler, IRecordPositionHandler { protected abstract DataSourceType getSourceType(); } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/impl/HttpPositionHandler.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/impl/HttpPositionHandler.java index b8d536f388..a58fa31c07 100644 --- a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/impl/HttpPositionHandler.java +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/service/position/impl/HttpPositionHandler.java @@ -23,6 +23,7 @@ import org.apache.eventmesh.common.remote.datasource.DataSourceType; import org.apache.eventmesh.common.remote.offset.RecordPosition; import org.apache.eventmesh.common.remote.request.FetchPositionRequest; +import org.apache.eventmesh.common.remote.request.RecordPositionRequest; import org.apache.eventmesh.common.remote.request.ReportPositionRequest; import java.util.ArrayList; @@ -58,4 +59,9 @@ public List handler(FetchPositionRequest request, Metadata metad List recordPositionList = new ArrayList<>(); return recordPositionList; } + + @Override + public boolean handler(RecordPositionRequest request, Metadata metadata) { + return true; + } } 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 352ba57e96..8545078d80 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 @@ -21,18 +21,27 @@ import org.apache.eventmesh.admin.server.web.db.entity.EventMeshPositionReporterHistory; import org.apache.eventmesh.admin.server.web.db.service.EventMeshMysqlPositionService; import org.apache.eventmesh.admin.server.web.db.service.EventMeshPositionReporterHistoryService; +import org.apache.eventmesh.admin.server.web.pojo.BinlogPosition; +import org.apache.eventmesh.admin.server.web.pojo.JobDetail; +import org.apache.eventmesh.admin.server.web.service.job.JobInfoBizService; import org.apache.eventmesh.admin.server.web.service.position.PositionHandler; +import org.apache.eventmesh.admin.server.web.utils.JdbcUtils; +import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSourceConfig; +import org.apache.eventmesh.common.config.connector.rdb.canal.CanalSourceFullConfig; import org.apache.eventmesh.common.protocol.grpc.adminserver.Metadata; import org.apache.eventmesh.common.remote.datasource.DataSourceType; import org.apache.eventmesh.common.remote.offset.RecordPosition; import org.apache.eventmesh.common.remote.offset.canal.CanalRecordOffset; import org.apache.eventmesh.common.remote.offset.canal.CanalRecordPartition; import org.apache.eventmesh.common.remote.request.FetchPositionRequest; +import org.apache.eventmesh.common.remote.request.RecordPositionRequest; import org.apache.eventmesh.common.remote.request.ReportPositionRequest; import org.apache.eventmesh.common.utils.JsonUtils; import org.apache.commons.lang3.StringUtils; +import java.sql.PreparedStatement; +import java.sql.ResultSet; import java.time.Duration; import java.util.ArrayList; import java.util.List; @@ -42,6 +51,7 @@ import org.springframework.dao.DuplicateKeyException; import org.springframework.stereotype.Component; +import com.alibaba.druid.pool.DruidDataSource; import com.baomidou.mybatisplus.core.toolkit.Wrappers; import lombok.extern.slf4j.Slf4j; @@ -50,6 +60,12 @@ @Slf4j public class MysqlPositionHandler extends PositionHandler { private static final int RETRY_TIMES = 3; + private static final String SQL_SELECT_RDB_VERSION = "select version() as rdb_version"; + private static final String SQL_SHOW_BINLOG_POSITION = "SHOW MASTER STATUS"; + private static final String SQL_SELECT_SERVER_UUID_IN_MARIADB = "SELECT @@global.server_id as server_uuid"; + private static final String SQL_SHOW_SERVER_UUID_IN_MYSQL = "SELECT @@server_uuid as server_uuid"; + private static final String SQL_SELECT_GTID_IN_MARIADB = "SELECT @@global.gtid_binlog_pos as gtid"; + private static final String SQL_SELECT_GTID_IN_MYSQL = "SELECT @@gtid_executed as gtid"; private final long retryPeriod = Duration.ofMillis(500).toNanos(); @@ -59,6 +75,9 @@ public class MysqlPositionHandler extends PositionHandler { @Autowired EventMeshPositionReporterHistoryService historyService; + @Autowired + JobInfoBizService jobInfoBizService; + @Override protected DataSourceType getSourceType() { return DataSourceType.MYSQL; @@ -67,8 +86,8 @@ protected DataSourceType getSourceType() { private boolean isNotForward(EventMeshMysqlPosition now, EventMeshMysqlPosition old) { if (StringUtils.isNotBlank(old.getJournalName()) && old.getJournalName().equals(now.getJournalName()) && old.getPosition() >= now.getPosition()) { - log.info("job [{}] report position [{}] by runtime [{}] less than db position [{}] journal name [{}] by [{}]", - now.getJobID(), now.getPosition(), now.getAddress(), now.getJournalName(), old.getPosition(), old.getAddress()); + log.info("job [{}] report position [{}] by runtime [{}] less than db position [{}] journal name [{}] by [{}]", now.getJobID(), + now.getPosition(), now.getAddress(), now.getJournalName(), old.getPosition(), old.getAddress()); return true; } return false; @@ -76,8 +95,7 @@ private boolean isNotForward(EventMeshMysqlPosition now, EventMeshMysqlPosition public boolean saveOrUpdateByJob(EventMeshMysqlPosition position) { for (int i = 0; i < RETRY_TIMES; i++) { - EventMeshMysqlPosition old = positionService.getOne(Wrappers.query().eq("jobId", - position.getJobID())); + EventMeshMysqlPosition old = positionService.getOne(Wrappers.query().eq("jobId", position.getJobID())); if (old == null) { try { return positionService.save(position); @@ -95,8 +113,8 @@ public boolean saveOrUpdateByJob(EventMeshMysqlPosition position) { return true; } try { - if (!positionService.update(position, Wrappers.update().eq("updateTime", - old.getUpdateTime()).eq("jobID", old.getJobID()))) { + if (!positionService.update(position, + Wrappers.update().eq("updateTime", old.getUpdateTime()).eq("jobID", old.getJobID()))) { log.warn("update position [{}] fail, maybe current update. it will retry in 500ms", position); LockSupport.parkNanos(retryPeriod); continue; @@ -123,7 +141,6 @@ public boolean saveOrUpdateByJob(EventMeshMysqlPosition position) { @Override public boolean handler(ReportPositionRequest request, Metadata metadata) { - try { List recordPositionList = request.getRecordPositionList(); RecordPosition recordPosition = recordPositionList.get(0); @@ -170,8 +187,7 @@ public boolean handler(ReportPositionRequest request, Metadata metadata) { @Override public List handler(FetchPositionRequest request, Metadata metadata) { - List positionList = positionService.list(Wrappers.query().eq("jobID", - request.getJobID())); + List positionList = positionService.list(Wrappers.query().eq("jobID", request.getJobID())); List recordPositionList = new ArrayList<>(); for (EventMeshMysqlPosition position : positionList) { CanalRecordPartition partition = new CanalRecordPartition(); @@ -189,4 +205,147 @@ public List handler(FetchPositionRequest request, Metadata metad } return recordPositionList; } + + @Override + public boolean handler(RecordPositionRequest request, Metadata metadata) { + try { + String fullJobID = request.getFullJobID(); + String increaseJobID = request.getIncreaseJobID(); + log.info("start record full job position to increase job position,full jobID:{}, increase jobID:{}.", fullJobID, increaseJobID); + JobDetail fullJobDetail = jobInfoBizService.getJobDetail(fullJobID); + CanalSourceConfig canalSourceConfig = (CanalSourceConfig) fullJobDetail.getSourceDataSource().getConf(); + CanalSourceFullConfig canalSourceFullConfig = JsonUtils.mapToObject(canalSourceConfig.getSourceConfig(), CanalSourceFullConfig.class); + try (DruidDataSource druidDataSource = JdbcUtils.createDruidDataSource(canalSourceFullConfig.getSourceConnectorConfig().getUrl(), + canalSourceFullConfig.getSourceConnectorConfig().getUserName(), canalSourceFullConfig.getSourceConnectorConfig().getPassWord())) { + + DataSourceType dataSourceType = checkRDBDataSourceType(druidDataSource); + + ReportPositionRequest reportPositionRequest = new ReportPositionRequest(); + reportPositionRequest.setJobID(increaseJobID); + reportPositionRequest.setDataSourceType(DataSourceType.MYSQL); + reportPositionRequest.setAddress(request.getAddress()); + + RecordPosition recordPosition = new RecordPosition(); + CanalRecordOffset recordOffset = new CanalRecordOffset(); + BinlogPosition binlogPosition = queryBinlogPosition(druidDataSource); + String gtid = queryGTID(druidDataSource, dataSourceType); + recordOffset.setOffset(binlogPosition.getPosition()); + recordOffset.setGtid(gtid); + recordPosition.setRecordOffset(recordOffset); + + CanalRecordPartition recordPartition = new CanalRecordPartition(); + String serverUUID = queryServerUUID(druidDataSource, dataSourceType); + recordPartition.setJournalName(binlogPosition.getFile()); + recordPartition.setServerUUID(serverUUID); + recordPosition.setRecordPartition(recordPartition); + + List recordPositions = new ArrayList<>(); + recordPositions.add(recordPosition); + + reportPositionRequest.setRecordPositionList(recordPositions); + log.info("start store increase task position,jobID:{},request:{}", increaseJobID, reportPositionRequest); + handler(reportPositionRequest, metadata); + } + return true; + } catch (Exception e) { + log.error("record full job position to increase job position failed.", e); + return false; + } + } + + private DataSourceType checkRDBDataSourceType(DruidDataSource druidDataSource) { + try { + log.info("execute sql '{}' start.", SQL_SELECT_RDB_VERSION); + try (PreparedStatement preparedStatement = druidDataSource.getConnection().prepareStatement(SQL_SELECT_RDB_VERSION)) { + ResultSet resultSet = preparedStatement.executeQuery(); + if (resultSet.next()) { + log.info("execute sql '{}' result:{}", SQL_SELECT_RDB_VERSION, resultSet); + String rdbVersion = resultSet.getString("rdb_version"); + if (StringUtils.isNotBlank(rdbVersion)) { + if (rdbVersion.toLowerCase().contains(DataSourceType.MariaDB.getName().toLowerCase())) { + return DataSourceType.MariaDB; + } + } + } + } + } catch (Exception e) { + log.warn("select rdb version failed,data source:{}", druidDataSource, e); + throw new RuntimeException("select rdb version failed"); + } + return DataSourceType.MYSQL; + } + + private BinlogPosition queryBinlogPosition(DruidDataSource druidDataSource) { + BinlogPosition binlogPosition = new BinlogPosition(); + try { + log.info("execute sql '{}' start.", SQL_SHOW_BINLOG_POSITION); + try (PreparedStatement preparedStatement = druidDataSource.getConnection().prepareStatement(SQL_SHOW_BINLOG_POSITION)) { + ResultSet resultSet = preparedStatement.executeQuery(); + if (resultSet.next()) { + log.info("execute sql '{}' result:{}", SQL_SELECT_RDB_VERSION, resultSet); + String fileName = resultSet.getString("File"); + Long position = resultSet.getLong("Position"); + binlogPosition.setFile(fileName); + binlogPosition.setPosition(position); + } + } + } catch (Exception e) { + log.warn("show binlog position failed,data source:{}", druidDataSource, e); + throw new RuntimeException("show binlog position failed"); + } + return binlogPosition; + } + + private String queryServerUUID(DruidDataSource druidDataSource, DataSourceType dataSourceType) { + String serverUUID = ""; + try { + String queryServerUUIDSql; + if (DataSourceType.MariaDB.equals(dataSourceType)) { + queryServerUUIDSql = SQL_SELECT_SERVER_UUID_IN_MARIADB; + } else { + queryServerUUIDSql = SQL_SHOW_SERVER_UUID_IN_MYSQL; + } + log.info("execute sql '{}' start.", queryServerUUIDSql); + try (PreparedStatement preparedStatement = druidDataSource.getConnection().prepareStatement(queryServerUUIDSql)) { + ResultSet resultSet = preparedStatement.executeQuery(); + if (resultSet.next()) { + log.info("execute sql '{}' result:{}", queryServerUUIDSql, resultSet); + serverUUID = resultSet.getString("server_uuid"); + log.info("execute sql '{}',query server_uuid result:{}", queryServerUUIDSql, serverUUID); + return serverUUID; + } + } + } catch (Exception e) { + log.warn("select server_uuid failed,data source:{}", druidDataSource, e); + throw new RuntimeException("select server_uuid failed"); + } + return serverUUID; + } + + private String queryGTID(DruidDataSource druidDataSource, DataSourceType dataSourceType) { + String gitd = ""; + try { + String queryGTIDSql; + if (DataSourceType.MariaDB.equals(dataSourceType)) { + queryGTIDSql = SQL_SELECT_GTID_IN_MARIADB; + } else { + queryGTIDSql = SQL_SELECT_GTID_IN_MYSQL; + } + log.info("execute sql '{}' start.", queryGTIDSql); + try (PreparedStatement preparedStatement = druidDataSource.getConnection().prepareStatement(queryGTIDSql)) { + ResultSet resultSet = preparedStatement.executeQuery(); + if (resultSet.next()) { + log.info("execute sql '{}' result:{}", queryGTIDSql, resultSet); + gitd = resultSet.getString("gtid"); + log.info("execute sql '{}',select gitd result:{}", queryGTIDSql, gitd); + return gitd; + } + } + } catch (Exception e) { + log.warn("select gtid failed,data source:{}", druidDataSource, e); + // when db server not open gitd mode, ignore gtid query exception + //throw new RuntimeException("select gtid failed"); + } + return gitd; + } } 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 7bc16ba4ac..d3c7087d47 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 @@ -18,25 +18,33 @@ package org.apache.eventmesh.admin.server.web.service.task; import org.apache.eventmesh.admin.server.AdminServerProperties; -import org.apache.eventmesh.admin.server.web.Response; import org.apache.eventmesh.admin.server.web.db.entity.EventMeshJobInfo; 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; import org.apache.eventmesh.admin.server.web.service.job.JobInfoBizService; import org.apache.eventmesh.common.config.connector.Config; +import org.apache.eventmesh.common.exception.EventMeshException; +import org.apache.eventmesh.common.remote.JobState; import org.apache.eventmesh.common.remote.TaskState; import org.apache.eventmesh.common.remote.datasource.DataSource; import org.apache.eventmesh.common.remote.datasource.DataSourceType; import org.apache.eventmesh.common.remote.request.CreateTaskRequest; +import org.apache.eventmesh.common.remote.request.QueryTaskInfoRequest; +import org.apache.eventmesh.common.remote.request.TaskBachRequest; +import org.apache.eventmesh.common.remote.request.TaskIDRequest; import org.apache.eventmesh.common.remote.response.CreateTaskResponse; +import org.apache.eventmesh.common.remote.response.QueryTaskInfoResponse; +import org.apache.eventmesh.common.remote.response.SimpleResponse; import org.apache.eventmesh.common.utils.JsonUtils; import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Random; import java.util.UUID; import java.util.stream.Collectors; @@ -47,6 +55,11 @@ import org.springframework.transaction.annotation.Transactional; import org.springframework.web.client.RestTemplate; +import com.baomidou.mybatisplus.core.toolkit.Wrappers; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j @Service public class TaskBizService { @@ -81,7 +94,7 @@ public CreateTaskResponse createTask(CreateTaskRequest req) { String remoteResponse = ""; // not from other admin && target not equals with self region if (!req.isFlag() && !properties.getRegion().equals(targetRegion)) { - List adminServerList = properties.getAdminServerList().get(targetRegion); + List adminServerList = Arrays.asList(properties.getAdminServerList().get(targetRegion).split(";")); if (adminServerList == null || adminServerList.isEmpty()) { throw new RuntimeException("No admin server available for region: " + targetRegion); } @@ -165,11 +178,183 @@ private CreateTaskResponse buildCreateTaskResponse(String taskId, Listquery() + .eq("taskID", taskIDRequest.getTaskID())); + + if (Objects.isNull(taskInfoServiceOne)) { + throw new EventMeshException("task not found"); + } + + if (TaskState.DELETE.name().equals(taskInfoServiceOne.getTaskState())) { + throw new EventMeshException("task already deleted"); + } + + // update task state + taskInfoService.update(Wrappers.update() + .eq("id", taskInfoServiceOne.getId()) + .set("taskState", TaskState.RUNNING.name())); + + List eventMeshJobInfos = jobInfoService.getJobsByTaskID(taskIDRequest.getTaskID()); + + for (EventMeshJobInfo eventMeshJobInfo : eventMeshJobInfos) { + // update job state by jonID + jobInfoService.updateJobState(eventMeshJobInfo.getJobID(), JobState.RUNNING); + } + + // todo: start task job eventmesh-runtime-v2 schedule ? + + } catch (Exception e) { + log.info("start task exception:{}", e.getMessage()); + throw new EventMeshException("start task exception"); + } + } + + @Transactional + public boolean deleteTaskByTaskID(TaskIDRequest taskIDRequest) { + try { + EventMeshTaskInfo taskInfoServiceOne = taskInfoService.getOne(Wrappers.query() + .eq("taskID", taskIDRequest.getTaskID())); + + if (Objects.isNull(taskInfoServiceOne)) { + throw new EventMeshException("task not found"); + } + + if (!TaskState.DELETE.name().equals(taskInfoServiceOne.getTaskState())) { + // update task state to delete + taskInfoService.update(Wrappers.update() + .eq("id", taskInfoServiceOne.getId()) + .set("taskState", TaskState.DELETE.name())); + } + List eventMeshJobInfos = jobInfoService.getJobsByTaskID(taskInfoServiceOne.getTaskID()); + for (EventMeshJobInfo eventMeshJobInfo : eventMeshJobInfos) { + // update job state to delete + jobInfoService.updateJobState(eventMeshJobInfo.getJobID(), JobState.DELETE); + } + // todo: data source config need delete? + + } catch (RuntimeException e) { + log.error("delete task failed:{}", e.getMessage()); + throw new EventMeshException("delete task failed"); + } + return true; + } + + public List queryTaskInfo(QueryTaskInfoRequest taskInfoRequest) { + return taskInfoService.queryTaskInfo(taskInfoRequest); + } + + @Transactional + public void restartTask(TaskIDRequest taskIDRequest) { + try { + EventMeshTaskInfo taskInfoServiceOne = taskInfoService.getOne(Wrappers.query() + .eq("taskID", taskIDRequest.getTaskID()) + .ne("taskState", TaskState.DELETE.name())); + + if (Objects.isNull(taskInfoServiceOne)) { + throw new EventMeshException("task not found"); + } + if (!TaskState.RUNNING.name().equals(taskInfoServiceOne.getTaskState())) { + taskInfoService.update(Wrappers.update() + .eq("id", taskInfoServiceOne.getId()) + .set("taskState", TaskState.RUNNING.name())); + } + List eventMeshJobInfos = jobInfoService.getJobsByTaskID(taskInfoServiceOne.getTaskID()); + for (EventMeshJobInfo eventMeshJobInfo : eventMeshJobInfos) { + // update job state to restart + jobInfoService.updateJobState(eventMeshJobInfo.getJobID(), JobState.RUNNING); + } + // todo: start task job eventmesh-runtime-v2 schedule? + + } catch (RuntimeException e) { + log.error("restart task filed:{}", e.getMessage()); + throw new EventMeshException("restart task filed"); + } + } + + @Transactional + public void stopTask(TaskIDRequest taskIDRequest) { + try { + EventMeshTaskInfo taskInfoServiceOne = taskInfoService.getOne(Wrappers.query() + .eq("taskID", taskIDRequest.getTaskID())); + + if (Objects.isNull(taskInfoServiceOne)) { + throw new EventMeshException("task not found"); + } + if (!TaskState.PAUSE.name().equals(taskInfoServiceOne.getTaskState())) { + taskInfoService.update(Wrappers.update() + .eq("id", taskInfoServiceOne.getId()) + .set("taskState", TaskState.PAUSE.name())); + } + + List eventMeshJobInfos = jobInfoService.getJobsByTaskID(taskInfoServiceOne.getTaskID()); + for (EventMeshJobInfo eventMeshJobInfo : eventMeshJobInfos) { + // update job state to pause + jobInfoService.updateJobState(eventMeshJobInfo.getJobID(), JobState.PAUSE); + } + + // todo: stop task job eventmesh-runtime-v2 schedule? + + } catch (RuntimeException e) { + log.error("stop task filed:{}", e.getMessage()); + throw new EventMeshException("stop task filed"); + } + } + + @Transactional + public void restartBatchTask(List taskIDRequestList, List errorNames) { + for (TaskBachRequest task : taskIDRequestList) { + try { + TaskIDRequest taskIDRequest = new TaskIDRequest(); + taskIDRequest.setTaskID(task.getTaskID()); + startTask(taskIDRequest); + } catch (RuntimeException e) { + log.error("restart batch task failed:{}", e.getMessage()); + errorNames.add(task.getTaskName()); + } + } + } + + @Transactional + public void stopBatchTask(List taskIDRequestList, List errorNames) { + for (TaskBachRequest task : taskIDRequestList) { + try { + TaskIDRequest taskIDRequest = new TaskIDRequest(); + taskIDRequest.setTaskID(task.getTaskID()); + stopTask(taskIDRequest); + } catch (RuntimeException e) { + log.error("stop batch task failed:{}", e.getMessage()); + errorNames.add(task.getTaskName()); + } + } + } + + @Transactional + public void startBatchTask(List taskIDRequestList, List errorNames) { + for (TaskBachRequest task : taskIDRequestList) { + try { + TaskIDRequest taskIDRequest = new TaskIDRequest(); + taskIDRequest.setTaskID(task.getTaskID()); + restartTask(taskIDRequest); + } catch (RuntimeException e) { + log.error("start batch task failed:{}", e.getMessage()); + errorNames.add(task.getTaskName()); + } + } + } + } diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/Base64.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/Base64.java new file mode 100644 index 0000000000..f85807b7f9 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/Base64.java @@ -0,0 +1,125 @@ +/* + * 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.utils; + +public class Base64 { + private static char[] alphabet = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/=".toCharArray(); + private static byte[] codes = new byte[256]; + + public Base64() { + } + + public static char[] encode(byte[] data) { + char[] out = new char[(data.length + 2) / 3 * 4]; + int i = 0; + + for (int index = 0; i < data.length; index += 4) { + boolean quad = false; + boolean trip = false; + int val = 255 & data[i]; + val <<= 8; + if (i + 1 < data.length) { + val |= 255 & data[i + 1]; + trip = true; + } + + val <<= 8; + if (i + 2 < data.length) { + val |= 255 & data[i + 2]; + quad = true; + } + + out[index + 3] = alphabet[quad ? val & 63 : 64]; + val >>= 6; + out[index + 2] = alphabet[trip ? val & 63 : 64]; + val >>= 6; + out[index + 1] = alphabet[val & 63]; + val >>= 6; + out[index + 0] = alphabet[val & 63]; + i += 3; + } + + return out; + } + + public static byte[] decode(char[] data) { + int tempLen = data.length; + + int len; + for (len = 0; len < data.length; ++len) { + if (data[len] > 255 || codes[data[len]] < 0) { + --tempLen; + } + } + + len = tempLen / 4 * 3; + if (tempLen % 4 == 3) { + len += 2; + } + + if (tempLen % 4 == 2) { + ++len; + } + + byte[] out = new byte[len]; + int shift = 0; + int accum = 0; + int index = 0; + + for (int ix = 0; ix < data.length; ++ix) { + int value = data[ix] > 255 ? -1 : codes[data[ix]]; + if (value >= 0) { + accum <<= 6; + shift += 6; + accum |= value; + if (shift >= 8) { + shift -= 8; + out[index++] = (byte) (accum >> shift & 255); + } + } + } + + if (index != out.length) { + throw new Error("Miscalculated data length (wrote " + index + " instead of " + out.length + ")"); + } else { + return out; + } + } + + static { + int i; + for (i = 0; i < 256; ++i) { + codes[i] = -1; + } + + for (i = 65; i <= 90; ++i) { + codes[i] = (byte) (i - 65); + } + + for (i = 97; i <= 122; ++i) { + codes[i] = (byte) (26 + i - 97); + } + + for (i = 48; i <= 57; ++i) { + codes[i] = (byte) (52 + i - 48); + } + + codes[43] = 62; + codes[47] = 63; + } +} diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/Base64Utils.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/Base64Utils.java new file mode 100644 index 0000000000..9c9a258671 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/Base64Utils.java @@ -0,0 +1,94 @@ +/* + * 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.utils; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.InputStream; +import java.io.OutputStream; + +public class Base64Utils { + private static final int CACHE_SIZE = 1024; + + public Base64Utils() { + } + + public static byte[] decode(String base64) throws Exception { + return Base64.decode(base64.toCharArray()); + } + + public static String encode(byte[] bytes) throws Exception { + return new String(Base64.encode(bytes)); + } + + public static String encodeFile(String filePath) throws Exception { + byte[] bytes = fileToByte(filePath); + return encode(bytes); + } + + public static void decodeToFile(String filePath, String base64) throws Exception { + byte[] bytes = decode(base64); + byteArrayToFile(bytes, filePath); + } + + public static byte[] fileToByte(String filePath) throws Exception { + byte[] data = new byte[0]; + File file = new File(filePath); + if (file.exists()) { + FileInputStream in = new FileInputStream(file); + ByteArrayOutputStream out = new ByteArrayOutputStream(2048); + byte[] cache = new byte[1024]; + int nread; + + while ((nread = in.read(cache)) != -1) { + out.write(cache, 0, nread); + out.flush(); + } + + out.close(); + in.close(); + data = out.toByteArray(); + } + + return data; + } + + public static void byteArrayToFile(byte[] bytes, String filePath) throws Exception { + InputStream in = new ByteArrayInputStream(bytes); + File destFile = new File(filePath); + if (!destFile.getParentFile().exists()) { + destFile.getParentFile().mkdirs(); + } + + destFile.createNewFile(); + OutputStream out = new FileOutputStream(destFile); + byte[] cache = new byte[1024]; + + int nread; + while ((nread = ((InputStream) in).read(cache)) != -1) { + ((OutputStream) out).write(cache, 0, nread); + ((OutputStream) out).flush(); + } + + ((OutputStream) out).close(); + ((InputStream) in).close(); + } +} diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/EncryptUtil.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/EncryptUtil.java new file mode 100644 index 0000000000..06c8bbc330 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/EncryptUtil.java @@ -0,0 +1,138 @@ +/* + * 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.utils; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; + +public class EncryptUtil { + public EncryptUtil() { + } + + private static byte[] hexStringToBytes(String hexString) { + if (hexString != null && !hexString.equals("")) { + hexString = hexString.toUpperCase(); + int length = hexString.length() / 2; + char[] hexChars = hexString.toCharArray(); + byte[] d = new byte[length]; + + for (int i = 0; i < length; ++i) { + int pos = i * 2; + d[i] = (byte) (charToByte(hexChars[pos]) << 4 | charToByte(hexChars[pos + 1])); + } + + return d; + } else { + return null; + } + } + + public static String byteToHexString(byte[] b) { + String a = ""; + + for (int i = 0; i < b.length; ++i) { + String hex = Integer.toHexString(b[i] & 255); + if (hex.length() == 1) { + hex = '0' + hex; + } + + a = a + hex; + } + + return a; + } + + private static byte charToByte(char c) { + return (byte) "0123456789ABCDEF".indexOf(c); + } + + private static String readFileContent(String filePath) { + File file = new File(filePath); + BufferedReader reader = null; + StringBuffer key = new StringBuffer(); + + try { + IOException e; + try { + reader = new BufferedReader(new FileReader(file)); + e = null; + + String tempString; + while ((tempString = reader.readLine()) != null) { + if (!tempString.startsWith("--")) { + key.append(tempString); + } + } + + reader.close(); + } catch (IOException ioException) { + e = ioException; + e.printStackTrace(); + } + } finally { + if (reader != null) { + try { + reader.close(); + } catch (IOException ioException) { + ioException.printStackTrace(); + } + } + + } + + return key.toString(); + } + + public static String decrypt(String sysPubKeyFile, String appPrivKeyFile, String encStr) throws Exception { + String pubKeyBase64 = readFileContent(sysPubKeyFile); + String privKeyBase64 = readFileContent(appPrivKeyFile); + byte[] encBin = hexStringToBytes(encStr); + byte[] pubDecBin = RSAUtils.decryptByPublicKeyBlock(encBin, pubKeyBase64); + byte[] privDecBin = RSAUtils.decryptByPrivateKeyBlock(pubDecBin, privKeyBase64); + return new String(privDecBin); + } + + public static String decrypt(ParamType pubKeyType, String sysPubKey, ParamType privKeyType, String appPrivKey, ParamType passwdType, + String passwd) throws Exception { + String pubKeyBase64 = pubKeyType == ParamType.FILE ? readFileContent(sysPubKey) : sysPubKey; + String privKeyBase64 = privKeyType == ParamType.FILE ? readFileContent(appPrivKey) : appPrivKey; + String passwdContent = passwdType == ParamType.FILE ? readFileContent(passwd) : passwd; + byte[] encBin = hexStringToBytes(passwdContent); + byte[] pubDecBin = RSAUtils.decryptByPublicKeyBlock(encBin, pubKeyBase64); + byte[] privDecBin = RSAUtils.decryptByPrivateKeyBlock(pubDecBin, privKeyBase64); + return new String(privDecBin); + } + + public static String encrypt(String appPubKeyFile, String sysPrivKeyFile, String passwd) throws Exception { + String pubKeyBase64 = readFileContent(appPubKeyFile); + String privKeyBase64 = readFileContent(sysPrivKeyFile); + byte[] pubEncBin = RSAUtils.encryptByPublicKeyBlock(passwd.getBytes(), pubKeyBase64); + byte[] privEncBin = RSAUtils.encryptByPrivateKeyBlock(pubEncBin, privKeyBase64); + return byteToHexString(privEncBin); + } + + public static String encrypt(ParamType pubKeyType, String appPubKey, ParamType privKeyType, String sysPrivKey, String passwd) throws Exception { + String pubKeyBase64 = pubKeyType == ParamType.FILE ? readFileContent(appPubKey) : appPubKey; + String privKeyBase64 = privKeyType == ParamType.FILE ? readFileContent(sysPrivKey) : sysPrivKey; + byte[] pubEncBin = RSAUtils.encryptByPublicKeyBlock(passwd.getBytes(), pubKeyBase64); + byte[] privEncBin = RSAUtils.encryptByPrivateKeyBlock(pubEncBin, privKeyBase64); + return byteToHexString(privEncBin); + } +} diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/JdbcUtils.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/JdbcUtils.java new file mode 100644 index 0000000000..c012806e2e --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/JdbcUtils.java @@ -0,0 +1,43 @@ +/* + * 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.utils; + +import com.alibaba.druid.pool.DruidDataSource; + +public class JdbcUtils { + + public static DruidDataSource createDruidDataSource(String url, String userName, String passWord) { + DruidDataSource dataSource = new DruidDataSource(); + dataSource.setUrl(url); + dataSource.setUsername(userName); + dataSource.setPassword(passWord); + dataSource.setInitialSize(5); + dataSource.setMinIdle(5); + dataSource.setMaxActive(20); + dataSource.setMaxWait(60000); + dataSource.setTimeBetweenEvictionRunsMillis(60000); + dataSource.setMinEvictableIdleTimeMillis(300000); + dataSource.setValidationQuery("SELECT 1"); + dataSource.setTestWhileIdle(true); + dataSource.setTestOnBorrow(false); + dataSource.setTestOnReturn(false); + dataSource.setPoolPreparedStatements(true); + dataSource.setMaxPoolPreparedStatementPerConnectionSize(20); + return dataSource; + } +} diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/ParamType.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/ParamType.java new file mode 100644 index 0000000000..ed58a49b89 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/ParamType.java @@ -0,0 +1,26 @@ +/* + * 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.utils; + +public enum ParamType { + FILE, + STRING; + + private ParamType() { + } +} diff --git a/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/RSAUtils.java b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/RSAUtils.java new file mode 100644 index 0000000000..9353eb3f17 --- /dev/null +++ b/eventmesh-admin-server/src/main/java/org/apache/eventmesh/admin/server/web/utils/RSAUtils.java @@ -0,0 +1,255 @@ +/* + * 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.utils; + +import java.io.ByteArrayOutputStream; +import java.security.Key; +import java.security.KeyFactory; +import java.security.KeyPair; +import java.security.KeyPairGenerator; +import java.security.PrivateKey; +import java.security.PublicKey; +import java.security.Signature; +import java.security.interfaces.RSAPrivateKey; +import java.security.interfaces.RSAPublicKey; +import java.security.spec.PKCS8EncodedKeySpec; +import java.security.spec.X509EncodedKeySpec; +import java.util.HashMap; +import java.util.Map; + +import javax.crypto.Cipher; + +public class RSAUtils { + public static final String KEY_ALGORITHM = "RSA"; + public static final String SIGNATURE_ALGORITHM = "MD5withRSA"; + private static final String PUBLIC_KEY = "RSAPublicKey"; + private static final String PRIVATE_KEY = "RSAPrivateKey"; + private static final int MAX_ENCRYPT_BLOCK = 117; + private static final int MAX_DECRYPT_BLOCK = 128; + + public RSAUtils() { + } + + public static Map genKeyPair() throws Exception { + KeyPairGenerator keyPairGen = KeyPairGenerator.getInstance("RSA"); + keyPairGen.initialize(1024); + KeyPair keyPair = keyPairGen.generateKeyPair(); + RSAPublicKey publicKey = (RSAPublicKey) keyPair.getPublic(); + RSAPrivateKey privateKey = (RSAPrivateKey) keyPair.getPrivate(); + Map keyMap = new HashMap(2); + keyMap.put("RSAPublicKey", publicKey); + keyMap.put("RSAPrivateKey", privateKey); + return keyMap; + } + + public static String sign(byte[] data, String privateKey) throws Exception { + byte[] keyBytes = Base64Utils.decode(privateKey); + PKCS8EncodedKeySpec pkcs8KeySpec = new PKCS8EncodedKeySpec(keyBytes); + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + PrivateKey privateK = keyFactory.generatePrivate(pkcs8KeySpec); + Signature signature = Signature.getInstance("MD5withRSA"); + signature.initSign(privateK); + signature.update(data); + return Base64Utils.encode(signature.sign()); + } + + public static boolean verify(byte[] data, String publicKey, String sign) throws Exception { + byte[] keyBytes = Base64Utils.decode(publicKey); + X509EncodedKeySpec keySpec = new X509EncodedKeySpec(keyBytes); + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + PublicKey publicK = keyFactory.generatePublic(keySpec); + Signature signature = Signature.getInstance("MD5withRSA"); + signature.initVerify(publicK); + signature.update(data); + return signature.verify(Base64Utils.decode(sign)); + } + + public static byte[] decryptByPrivateKey(byte[] encryptedData, String privateKey) throws Exception { + byte[] keyBytes = Base64Utils.decode(privateKey); + PKCS8EncodedKeySpec pkcs8KeySpec = new PKCS8EncodedKeySpec(keyBytes); + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + Key privateK = keyFactory.generatePrivate(pkcs8KeySpec); + Cipher cipher = Cipher.getInstance(keyFactory.getAlgorithm()); + cipher.init(2, privateK); + int inputLen = encryptedData.length; + ByteArrayOutputStream out = new ByteArrayOutputStream(); + int offSet = 0; + + for (int i = 0; inputLen - offSet > 0; offSet = i * 128) { + byte[] cache; + if (inputLen - offSet > 128) { + cache = cipher.doFinal(encryptedData, offSet, 128); + } else { + cache = cipher.doFinal(encryptedData, offSet, inputLen - offSet); + } + + out.write(cache, 0, cache.length); + ++i; + } + + byte[] decryptedData = out.toByteArray(); + out.close(); + return decryptedData; + } + + public static byte[] decryptByPrivateKeyBlock(byte[] encryptedData, String privateKey) throws Exception { + byte[] keyBytes = Base64Utils.decode(privateKey); + PKCS8EncodedKeySpec pkcs8KeySpec = new PKCS8EncodedKeySpec(keyBytes); + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + Key privateK = keyFactory.generatePrivate(pkcs8KeySpec); + Cipher cipher = Cipher.getInstance(keyFactory.getAlgorithm()); + cipher.init(2, privateK); + int inputLen = encryptedData.length; + int offSet = 0; + byte[] cache = cipher.doFinal(encryptedData, offSet, inputLen); + return cache; + } + + public static byte[] decryptByPublicKey(byte[] encryptedData, String publicKey) throws Exception { + byte[] keyBytes = Base64Utils.decode(publicKey); + X509EncodedKeySpec x509KeySpec = new X509EncodedKeySpec(keyBytes); + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + Key publicK = keyFactory.generatePublic(x509KeySpec); + Cipher cipher = Cipher.getInstance(keyFactory.getAlgorithm()); + cipher.init(2, publicK); + int inputLen = encryptedData.length; + ByteArrayOutputStream out = new ByteArrayOutputStream(); + int offSet = 0; + + for (int i = 0; inputLen - offSet > 0; offSet = i * 128) { + byte[] cache; + if (inputLen - offSet > 128) { + cache = cipher.doFinal(encryptedData, offSet, 128); + } else { + cache = cipher.doFinal(encryptedData, offSet, inputLen - offSet); + } + + out.write(cache, 0, cache.length); + ++i; + } + + byte[] decryptedData = out.toByteArray(); + out.close(); + return decryptedData; + } + + public static byte[] decryptByPublicKeyBlock(byte[] encryptedData, String publicKey) throws Exception { + byte[] keyBytes = Base64Utils.decode(publicKey); + X509EncodedKeySpec x509KeySpec = new X509EncodedKeySpec(keyBytes); + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + Key publicK = keyFactory.generatePublic(x509KeySpec); + Cipher cipher = Cipher.getInstance(keyFactory.getAlgorithm()); + cipher.init(2, publicK); + int inputLen = encryptedData.length; + int offSet = 0; + byte[] cache = cipher.doFinal(encryptedData, offSet, inputLen); + return cache; + } + + public static byte[] encryptByPublicKey(byte[] data, String publicKey) throws Exception { + byte[] keyBytes = Base64Utils.decode(publicKey); + X509EncodedKeySpec x509KeySpec = new X509EncodedKeySpec(keyBytes); + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + Key publicK = keyFactory.generatePublic(x509KeySpec); + Cipher cipher = Cipher.getInstance(keyFactory.getAlgorithm()); + cipher.init(1, publicK); + int inputLen = data.length; + ByteArrayOutputStream out = new ByteArrayOutputStream(); + int offSet = 0; + + for (int i = 0; inputLen - offSet > 0; offSet = i * 117) { + byte[] cache; + if (inputLen - offSet > 117) { + cache = cipher.doFinal(data, offSet, 117); + } else { + cache = cipher.doFinal(data, offSet, inputLen - offSet); + } + + out.write(cache, 0, cache.length); + ++i; + } + + byte[] encryptedData = out.toByteArray(); + out.close(); + return encryptedData; + } + + public static byte[] encryptByPublicKeyBlock(byte[] data, String publicKey) throws Exception { + byte[] keyBytes = Base64Utils.decode(publicKey); + X509EncodedKeySpec x509KeySpec = new X509EncodedKeySpec(keyBytes); + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + Key publicK = keyFactory.generatePublic(x509KeySpec); + Cipher cipher = Cipher.getInstance(keyFactory.getAlgorithm()); + cipher.init(1, publicK); + int inputLen = data.length; + int offSet = 0; + byte[] cache = cipher.doFinal(data, offSet, inputLen); + return cache; + } + + public static byte[] encryptByPrivateKey(byte[] data, String privateKey) throws Exception { + byte[] keyBytes = Base64Utils.decode(privateKey); + PKCS8EncodedKeySpec pkcs8KeySpec = new PKCS8EncodedKeySpec(keyBytes); + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + Key privateK = keyFactory.generatePrivate(pkcs8KeySpec); + Cipher cipher = Cipher.getInstance(keyFactory.getAlgorithm()); + cipher.init(1, privateK); + int inputLen = data.length; + ByteArrayOutputStream out = new ByteArrayOutputStream(); + int offSet = 0; + + for (int i = 0; inputLen - offSet > 0; offSet = i * 117) { + byte[] cache; + if (inputLen - offSet > 117) { + cache = cipher.doFinal(data, offSet, 117); + } else { + cache = cipher.doFinal(data, offSet, inputLen - offSet); + } + + out.write(cache, 0, cache.length); + ++i; + } + + byte[] encryptedData = out.toByteArray(); + out.close(); + return encryptedData; + } + + public static byte[] encryptByPrivateKeyBlock(byte[] data, String privateKey) throws Exception { + byte[] keyBytes = Base64Utils.decode(privateKey); + PKCS8EncodedKeySpec pkcs8KeySpec = new PKCS8EncodedKeySpec(keyBytes); + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + Key privateK = keyFactory.generatePrivate(pkcs8KeySpec); + Cipher cipher = Cipher.getInstance(keyFactory.getAlgorithm()); + cipher.init(1, privateK); + int inputLen = data.length; + int offSet = 0; + byte[] cache = cipher.doFinal(data, offSet, inputLen); + return cache; + } + + public static String getPrivateKey(Map keyMap) throws Exception { + Key key = (Key) keyMap.get("RSAPrivateKey"); + return Base64Utils.encode(key.getEncoded()); + } + + public static String getPublicKey(Map keyMap) throws Exception { + Key key = (Key) keyMap.get("RSAPublicKey"); + return Base64Utils.encode(key.getEncoded()); + } +} diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/JobState.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/JobState.java index da9daffe9c..150a67e302 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/JobState.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/JobState.java @@ -24,7 +24,7 @@ @ToString public enum JobState { - INIT, RUNNING, COMPLETE, DELETE, FAIL; + INIT, RUNNING, COMPLETE, DELETE, FAIL, PAUSE; private static final JobState[] STATES_NUM_INDEX = JobState.values(); private static final Map STATES_NAME_INDEX = new HashMap<>(); diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/QueryTaskInfoRequest.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/QueryTaskInfoRequest.java new file mode 100644 index 0000000000..c0973cf63d --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/QueryTaskInfoRequest.java @@ -0,0 +1,49 @@ +/* + * 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.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@AllArgsConstructor +@NoArgsConstructor +public class QueryTaskInfoRequest { + + private String taskDesc; + + private String taskID; + + private String jobType; + + private String sourceDataID; + + private String targetDataID; + + private String ip; + + private String sourceTableName; + + private String taskMathID; + + private Integer currentPage; + + private Integer pageSize; + +} diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/QueryTaskMonitorRequest.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/QueryTaskMonitorRequest.java new file mode 100644 index 0000000000..cd777d5019 --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/QueryTaskMonitorRequest.java @@ -0,0 +1,31 @@ +/* + * 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; +import lombok.ToString; + +@Data +@EqualsAndHashCode(callSuper = true) +@ToString +public class QueryTaskMonitorRequest extends BaseRemoteRequest { + private String taskID; + private String jobID; + private long limit; +} diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/RecordPositionRequest.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/RecordPositionRequest.java new file mode 100644 index 0000000000..b04a6f1041 --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/RecordPositionRequest.java @@ -0,0 +1,41 @@ +/* + * 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 org.apache.eventmesh.common.remote.JobState; +import org.apache.eventmesh.common.remote.datasource.DataSourceType; + +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.ToString; + +@Data +@EqualsAndHashCode(callSuper = true) +@ToString +public class RecordPositionRequest extends BaseRemoteRequest { + + private String fullJobID; + + private String increaseJobID; + // prepare to update job state to current state + private JobState updateState; + + private String address; + + private DataSourceType dataSourceType; +} diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/TaskBachRequest.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/TaskBachRequest.java new file mode 100644 index 0000000000..306badc156 --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/TaskBachRequest.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.common.remote.request; + +import lombok.Data; + +@Data +public class TaskBachRequest { + + private String taskID; + + private String taskName; + +} diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/TaskIDRequest.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/TaskIDRequest.java new file mode 100644 index 0000000000..37a72916ff --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/request/TaskIDRequest.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.common.remote.request; + +import lombok.Data; + +@Data +public class TaskIDRequest { + + private String taskID; + +} diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/BaseRemoteResponse.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/BaseRemoteResponse.java index 3ea8401535..84ea3661bd 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/BaseRemoteResponse.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/BaseRemoteResponse.java @@ -26,7 +26,7 @@ import lombok.Setter; @Getter -public abstract class BaseRemoteResponse implements IPayload { +public abstract class BaseRemoteResponse implements IPayload { @Setter private boolean success = true; @Setter @@ -35,6 +35,8 @@ public abstract class BaseRemoteResponse implements IPayload { private String desc; private Map header = new HashMap<>(); + @Setter + private T data; public void addHeader(String key, String value) { if (key == null || value == null) { diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/CreateTaskResponse.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/CreateTaskResponse.java index 11678dfcf0..24e7871e04 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/CreateTaskResponse.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/CreateTaskResponse.java @@ -24,7 +24,7 @@ import lombok.Data; @Data -public class CreateTaskResponse extends BaseRemoteResponse { +public class CreateTaskResponse { private String taskId; diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/HttpResponseResult.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/HttpResponseResult.java new file mode 100644 index 0000000000..b6ca8cef0d --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/HttpResponseResult.java @@ -0,0 +1,65 @@ +/* + * 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.response; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@NoArgsConstructor +@AllArgsConstructor +public class HttpResponseResult { + + private Integer code; + + private String message; + + private T data; + + public HttpResponseResult(Integer code, String message) { + this.code = code; + this.message = message; + } + + public HttpResponseResult(Integer code, T data) { + this.code = code; + this.data = data; + } + + public static HttpResponseResult success() { + return new HttpResponseResult<>(200, "success"); + } + + public static HttpResponseResult success(T data) { + return new HttpResponseResult<>(200, "success", data); + } + + public static HttpResponseResult failed() { + return new HttpResponseResult<>(500, "failed"); + } + + public static HttpResponseResult failed(T data) { + return new HttpResponseResult<>(500, "failed", data); + } + + public static HttpResponseResult exception(T data) { + return new HttpResponseResult<>(300, data); + } + +} \ No newline at end of file diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/QueryTaskInfoResponse.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/QueryTaskInfoResponse.java new file mode 100644 index 0000000000..4c0c536eae --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/QueryTaskInfoResponse.java @@ -0,0 +1,152 @@ +/* + * 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.response; + +import java.util.Date; +import java.util.List; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@AllArgsConstructor +@NoArgsConstructor +public class QueryTaskInfoResponse { + + // event_mesh_task_info + private Integer id; + + private String taskID; + + private String taskName; + + private String taskDesc; + + private String taskState; + + private String sourceRegion; + + private String targetRegion; + + private String createUid; + + private String updateUid; + + private Date createTime; + + private Date updateTime; + + List eventMeshJobInfoList; + + @Data + public static class EventMeshJobInfo { + // event_mesh_job_info + private Integer id; + + private String jobID; + + private String jobDesc; + + private String taskID; + + private String transportType; + + private Integer sourceData; + + private Integer targetData; + + 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; + + private Date createTime; + + private Date updateTime; + + // private List eventMeshDataSource; + + private EventMeshDataSource dataSource; + + private EventMeshDataSource dataSink; + + private EventMeshMysqlPosition eventMeshMysqlPosition; + + } + + @Data + public static class EventMeshDataSource { + + private Integer id; + + private String dataType; + + private String description; + + private String configuration; + + private String configurationClass; + + private String region; + + private String createUid; + + private String updateUid; + + private Date createTime; + + private Date updateTime; + } + + @Data + public static class EventMeshMysqlPosition { + + private Integer id; + + private String jobID; + + private String serverUUID; + + private String address; + + private Long position; + + private String gtid; + + private String currentGtid; + + private Long timestamp; + + private String journalName; + + private Date createTime; + + private Date updateTime; + } + +} \ No newline at end of file diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/QueryTaskMonitorResponse.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/QueryTaskMonitorResponse.java new file mode 100644 index 0000000000..432729a995 --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/response/QueryTaskMonitorResponse.java @@ -0,0 +1,31 @@ +/* + * 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.response; + +import org.apache.eventmesh.common.remote.task.TaskMonitor; + +import java.util.List; + +import lombok.Data; + +@Data +public class QueryTaskMonitorResponse { + + private List taskMonitors; + +} diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/task/TaskMonitor.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/task/TaskMonitor.java new file mode 100644 index 0000000000..6d303eec3b --- /dev/null +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/remote/task/TaskMonitor.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.eventmesh.common.remote.task; + +import java.io.Serializable; +import java.util.Date; + +import lombok.Data; + +@Data +public class TaskMonitor implements Serializable { + private String taskID; + private String jobID; + private String address; + private String transportType; + private String connectorStage; + private long totalReqNum; + private long totalTimeCost; + private long maxTimeCost; + private long avgTimeCost; + private double tps; + private Date createTime; + private static final long serialVersionUID = 1L; + +} From 30b27aeaedba10f2d8eded550af8318a5bfeb552 Mon Sep 17 00:00:00 2001 From: xwm1992 Date: Wed, 11 Dec 2024 20:42:58 +0800 Subject: [PATCH 5/5] [ISSUE #5144] update eventmesh-connector-http module --- .../connector/http/SinkConnectorConfig.java | 13 +++- .../connector/http/SourceConnectorConfig.java | 10 ++- .../common/SynchronizedCircularFifoQueue.java | 1 + .../http/sink/HttpSinkConnector.java | 44 ++++++++++++- .../http/sink/data/HttpExportMetadata.java | 2 - .../sink/handler/AbstractHttpSinkHandler.java | 64 ++++++++++++------- .../sink/handler/HttpDeliveryStrategy.java | 23 +++++++ .../handler/impl/CommonHttpSinkHandler.java | 8 ++- .../http/source/HttpSourceConnector.java | 38 ++++++----- .../protocol/impl/CloudEventProtocol.java | 2 +- 10 files changed, 153 insertions(+), 52 deletions(-) create mode 100644 eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/HttpDeliveryStrategy.java diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/http/SinkConnectorConfig.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/http/SinkConnectorConfig.java index ccebe5a998..65fc8fe72d 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/http/SinkConnectorConfig.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/http/SinkConnectorConfig.java @@ -39,8 +39,8 @@ public class SinkConnectorConfig { // timeunit: ms, default 5000ms private int idleTimeout = 5000; - // maximum number of HTTP/1 connections a client will pool, default 5 - private int maxConnectionPoolSize = 5; + // maximum number of HTTP/1 connections a client will pool, default 50 + private int maxConnectionPoolSize = 50; // retry config private HttpRetryConfig retryConfig = new HttpRetryConfig(); @@ -48,6 +48,15 @@ public class SinkConnectorConfig { // webhook config private HttpWebhookConfig webhookConfig = new HttpWebhookConfig(); + private String deliveryStrategy = "ROUND_ROBIN"; + + private boolean skipDeliverException = false; + + // managed pipelining param, default true + private boolean isParallelized = true; + + private int parallelism = 2; + /** * Fill default values if absent (When there are multiple default values for a field) diff --git a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/http/SourceConnectorConfig.java b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/http/SourceConnectorConfig.java index 282f883332..2c091e321a 100644 --- a/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/http/SourceConnectorConfig.java +++ b/eventmesh-common/src/main/java/org/apache/eventmesh/common/config/connector/http/SourceConnectorConfig.java @@ -44,12 +44,18 @@ public class SourceConnectorConfig { */ private int maxFormAttributeSize = 1024 * 1024; - // protocol, default Common + // max size of the queue, default 1000 + private int maxStorageSize = 1000; + + // batch size, default 10 + private int batchSize = 10; + + // protocol, default CloudEvent private String protocol = "Common"; // extra config, e.g. GitHub secret private Map extraConfig = new HashMap<>(); // data consistency enabled, default true - private boolean dataConsistencyEnabled = false; + private boolean dataConsistencyEnabled = true; } diff --git a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/common/SynchronizedCircularFifoQueue.java b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/common/SynchronizedCircularFifoQueue.java index 9989552d1e..0564e58734 100644 --- a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/common/SynchronizedCircularFifoQueue.java +++ b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/common/SynchronizedCircularFifoQueue.java @@ -142,6 +142,7 @@ public synchronized List fetchRange(int start, int end, boolean removed) { count++; } return items; + } 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 3df110f2e7..8e808ccc93 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 @@ -17,6 +17,7 @@ package org.apache.eventmesh.connector.http.sink; +import org.apache.eventmesh.common.EventMeshThreadFactory; import org.apache.eventmesh.common.config.connector.Config; import org.apache.eventmesh.common.config.connector.http.HttpSinkConfig; import org.apache.eventmesh.common.config.connector.http.SinkConnectorConfig; @@ -32,6 +33,10 @@ import java.util.List; import java.util.Objects; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import lombok.Getter; import lombok.SneakyThrows; @@ -45,6 +50,12 @@ public class HttpSinkConnector implements Sink, ConnectorCreateService { @Getter private HttpSinkHandler sinkHandler; + private ThreadPoolExecutor executor; + + private final LinkedBlockingQueue queue = new LinkedBlockingQueue<>(10000); + + private final AtomicBoolean isStart = new AtomicBoolean(true); + @Override public Class configClass() { return HttpSinkConfig.class; @@ -90,11 +101,30 @@ private void doInit() { } else { throw new IllegalArgumentException("Max retries must be greater than or equal to 0."); } + boolean isParallelized = this.httpSinkConfig.connectorConfig.isParallelized(); + int parallelism = isParallelized ? this.httpSinkConfig.connectorConfig.getParallelism() : 1; + executor = new ThreadPoolExecutor(parallelism, parallelism, 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(), new EventMeshThreadFactory("http-sink-handler")); } @Override public void start() throws Exception { this.sinkHandler.start(); + for (int i = 0; i < this.httpSinkConfig.connectorConfig.getParallelism(); i++) { + executor.execute(() -> { + while (isStart.get()) { + ConnectRecord connectRecord = null; + try { + connectRecord = queue.poll(2, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + if (connectRecord != null) { + sinkHandler.handle(connectRecord); + } + } + }); + } } @Override @@ -114,7 +144,18 @@ public void onException(ConnectRecord record) { @Override public void stop() throws Exception { + isStart.set(false); + while (!queue.isEmpty()) { + ConnectRecord record = queue.poll(); + this.sinkHandler.handle(record); + } + try { + Thread.sleep(50); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } this.sinkHandler.stop(); + log.info("All tasks completed, start shut down http sink connector"); } @Override @@ -125,8 +166,7 @@ public void put(List sinkRecords) { log.warn("ConnectRecord data is null, ignore."); continue; } - // Handle the ConnectRecord - this.sinkHandler.handle(sinkRecord); + queue.put(sinkRecord); } catch (Exception e) { log.error("Failed to sink message via HTTP. ", e); } diff --git a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/data/HttpExportMetadata.java b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/data/HttpExportMetadata.java index 41a5087870..111ee6b3e9 100644 --- a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/data/HttpExportMetadata.java +++ b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/data/HttpExportMetadata.java @@ -40,8 +40,6 @@ public class HttpExportMetadata implements Serializable { private LocalDateTime receivedTime; - private String httpRecordId; - private String recordId; private String retriedBy; diff --git a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/AbstractHttpSinkHandler.java b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/AbstractHttpSinkHandler.java index 28ba791127..9ef760617c 100644 --- a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/AbstractHttpSinkHandler.java +++ b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/AbstractHttpSinkHandler.java @@ -30,17 +30,26 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; +import lombok.Getter; + /** * AbstractHttpSinkHandler is an abstract class that provides a base implementation for HttpSinkHandler. */ public abstract class AbstractHttpSinkHandler implements HttpSinkHandler { + @Getter private final SinkConnectorConfig sinkConnectorConfig; + @Getter private final List urls; + private final HttpDeliveryStrategy deliveryStrategy; + + private int roundRobinIndex = 0; + protected AbstractHttpSinkHandler(SinkConnectorConfig sinkConnectorConfig) { this.sinkConnectorConfig = sinkConnectorConfig; + this.deliveryStrategy = HttpDeliveryStrategy.valueOf(sinkConnectorConfig.getDeliveryStrategy()); // Initialize URLs String[] urlStrings = sinkConnectorConfig.getUrls(); this.urls = Arrays.stream(urlStrings) @@ -48,14 +57,6 @@ protected AbstractHttpSinkHandler(SinkConnectorConfig sinkConnectorConfig) { .collect(Collectors.toList()); } - public SinkConnectorConfig getSinkConnectorConfig() { - return sinkConnectorConfig; - } - - public List getUrls() { - return urls; - } - /** * Processes a ConnectRecord by sending it over HTTP or HTTPS. This method should be called for each ConnectRecord that needs to be processed. * @@ -65,23 +66,38 @@ public List getUrls() { public void handle(ConnectRecord record) { // build attributes Map attributes = new ConcurrentHashMap<>(); - attributes.put(MultiHttpRequestContext.NAME, new MultiHttpRequestContext(urls.size())); - - // send the record to all URLs - for (URI url : urls) { - // convert ConnectRecord to HttpConnectRecord - String type = String.format("%s.%s.%s", - this.sinkConnectorConfig.getConnectorName(), url.getScheme(), - this.sinkConnectorConfig.getWebhookConfig().isActivate() ? "webhook" : "common"); - HttpConnectRecord httpConnectRecord = HttpConnectRecord.convertConnectRecord(record, type); - - // add AttemptEvent to the attributes - HttpAttemptEvent attemptEvent = new HttpAttemptEvent(this.sinkConnectorConfig.getRetryConfig().getMaxRetries() + 1); - attributes.put(HttpAttemptEvent.PREFIX + httpConnectRecord.getHttpRecordId(), attemptEvent); - - // deliver the record - deliver(url, httpConnectRecord, attributes, record); + + switch (deliveryStrategy) { + case ROUND_ROBIN: + attributes.put(MultiHttpRequestContext.NAME, new MultiHttpRequestContext(1)); + URI url = urls.get(roundRobinIndex); + roundRobinIndex = (roundRobinIndex + 1) % urls.size(); + sendRecordToUrl(record, attributes, url); + break; + case BROADCAST: + for (URI broadcastUrl : urls) { + attributes.put(MultiHttpRequestContext.NAME, new MultiHttpRequestContext(urls.size())); + sendRecordToUrl(record, attributes, broadcastUrl); + } + break; + default: + throw new IllegalArgumentException("Unknown delivery strategy: " + deliveryStrategy); } } + private void sendRecordToUrl(ConnectRecord record, Map attributes, URI url) { + // convert ConnectRecord to HttpConnectRecord + String type = String.format("%s.%s.%s", + this.sinkConnectorConfig.getConnectorName(), url.getScheme(), + this.sinkConnectorConfig.getWebhookConfig().isActivate() ? "webhook" : "common"); + HttpConnectRecord httpConnectRecord = HttpConnectRecord.convertConnectRecord(record, type); + + // add AttemptEvent to the attributes + HttpAttemptEvent attemptEvent = new HttpAttemptEvent(this.sinkConnectorConfig.getRetryConfig().getMaxRetries() + 1); + attributes.put(HttpAttemptEvent.PREFIX + httpConnectRecord.getHttpRecordId(), attemptEvent); + + // deliver the record + deliver(url, httpConnectRecord, attributes, record); + } + } diff --git a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/HttpDeliveryStrategy.java b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/HttpDeliveryStrategy.java new file mode 100644 index 0000000000..2e770eb120 --- /dev/null +++ b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/HttpDeliveryStrategy.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.connector.http.sink.handler; + +public enum HttpDeliveryStrategy { + ROUND_ROBIN, + BROADCAST +} diff --git a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/impl/CommonHttpSinkHandler.java b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/impl/CommonHttpSinkHandler.java index 61bdc9f310..0b57cc06ef 100644 --- a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/impl/CommonHttpSinkHandler.java +++ b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/sink/handler/impl/CommonHttpSinkHandler.java @@ -93,7 +93,8 @@ private void doInitWebClient() { .setIdleTimeout(sinkConnectorConfig.getIdleTimeout()) .setIdleTimeoutUnit(TimeUnit.MILLISECONDS) .setConnectTimeout(sinkConnectorConfig.getConnectionTimeout()) - .setMaxPoolSize(sinkConnectorConfig.getMaxConnectionPoolSize()); + .setMaxPoolSize(sinkConnectorConfig.getMaxConnectionPoolSize()) + .setPipelining(sinkConnectorConfig.isParallelized()); this.webClient = WebClient.create(vertx, options); } @@ -108,7 +109,7 @@ private void doInitWebClient() { */ @Override public Future> deliver(URI url, HttpConnectRecord httpConnectRecord, Map attributes, - ConnectRecord connectRecord) { + ConnectRecord connectRecord) { // create headers Map extensionMap = new HashMap<>(); Set extensionKeySet = httpConnectRecord.getExtensions().keySet(); @@ -203,6 +204,9 @@ private void tryCallback(HttpConnectRecord httpConnectRecord, Throwable e, Map queue; - private int maxBatchSize; - - private long maxPollWaitTime; + private int batchSize; private Route route; @@ -94,11 +94,11 @@ public void init(ConnectorContext connectorContext) { private void doInit() { // init queue - this.queue = new LinkedBlockingQueue<>(sourceConfig.getPollConfig().getCapacity()); + int maxQueueSize = this.sourceConfig.getConnectorConfig().getMaxStorageSize(); + this.queue = new LinkedBlockingQueue<>(maxQueueSize); - // init poll batch size and timeout - this.maxBatchSize = this.sourceConfig.getPollConfig().getMaxBatchSize(); - this.maxPollWaitTime = this.sourceConfig.getPollConfig().getMaxWaitTime(); + // init batch size + this.batchSize = this.sourceConfig.getConnectorConfig().getBatchSize(); // init protocol String protocolName = this.sourceConfig.getConnectorConfig().getProtocol(); @@ -136,14 +136,17 @@ public void start() { @Override public void commit(ConnectRecord record) { - if (this.route != null && sourceConfig.getConnectorConfig().isDataConsistencyEnabled()) { - this.route.handler(ctx -> { - // Return 200 OK - ctx.response() + if (sourceConfig.getConnectorConfig().isDataConsistencyEnabled()) { + log.debug("HttpSourceConnector commit record: {}", record.getRecordId()); + RoutingContext routingContext = (RoutingContext) record.getExtensionObj("routingContext"); + if (routingContext != null) { + routingContext.response() .putHeader("content-type", "application/json") .setStatusCode(HttpResponseStatus.OK.code()) - .end("{\"status\":\"success\",\"recordId\":\"" + record.getRecordId() + "\"}"); - }); + .end(CommonResponse.success().toJsonStr()); + } else { + log.error("Failed to commit the record, routingContext is null, recordId: {}", record.getRecordId()); + } } } @@ -185,13 +188,13 @@ public void stop() { @Override public List poll() { - // record current time long startTime = System.currentTimeMillis(); + long maxPollWaitTime = 5000; long remainingTime = maxPollWaitTime; // poll from queue - List connectRecords = new ArrayList<>(maxBatchSize); - for (int i = 0; i < maxBatchSize; i++) { + List connectRecords = new ArrayList<>(batchSize); + for (int i = 0; i < batchSize; i++) { try { Object obj = queue.poll(remainingTime, TimeUnit.MILLISECONDS); if (obj == null) { @@ -206,8 +209,9 @@ public List poll() { remainingTime = maxPollWaitTime > elapsedTime ? maxPollWaitTime - elapsedTime : 0; } catch (Exception e) { log.error("Failed to poll from queue.", e); - break; + throw new RuntimeException(e); } + } return connectRecords; } diff --git a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/source/protocol/impl/CloudEventProtocol.java b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/source/protocol/impl/CloudEventProtocol.java index a44ed0e90c..10158f6eba 100644 --- a/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/source/protocol/impl/CloudEventProtocol.java +++ b/eventmesh-connectors/eventmesh-connector-http/src/main/java/org/apache/eventmesh/connector/http/source/protocol/impl/CloudEventProtocol.java @@ -57,7 +57,7 @@ public void initialize(SourceConnectorConfig sourceConnectorConfig) { /** * Handle the protocol message for CloudEvent. * - * @param route route + * @param route route * @param queue queue info */ @Override