You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@eventmesh.apache.org by mi...@apache.org on 2022/10/23 14:45:23 UTC

[incubator-eventmesh] branch storage-api updated: [ISSUE #1784] add StorageConnectorService under storage api module

This is an automated email from the ASF dual-hosted git repository.

mikexue pushed a commit to branch storage-api
in repository https://gitbox.apache.org/repos/asf/incubator-eventmesh.git


The following commit(s) were added to refs/heads/storage-api by this push:
     new 1882fe22 [ISSUE #1784] add StorageConnectorService under storage api module
     new 8766ba17 Merge pull request #1785 from githublaohu/jdbc-connector-storage
1882fe22 is described below

commit 1882fe2218e8d46da0b66ba0cb6002b4b6a3a9bc
Author: githublaohu <23...@qq.com>
AuthorDate: Sun Oct 23 22:42:27 2022 +0800

    [ISSUE #1784] add StorageConnectorService under storage api module
---
 .../connector/storage/StorageConnectorInfo.java    |  26 ++++
 .../connector/storage/StorageConnectorProxy.java   | 164 +++++++++++++++++++++
 .../connector/storage/StorageConnectorService.java | 163 ++++++++++++++++++++
 3 files changed, 353 insertions(+)

diff --git a/eventmesh-connector-plugin/eventmesh-connector-api/src/main/java/org/apache/eventmesh/api/connector/storage/StorageConnectorInfo.java b/eventmesh-connector-plugin/eventmesh-connector-api/src/main/java/org/apache/eventmesh/api/connector/storage/StorageConnectorInfo.java
new file mode 100644
index 00000000..65a3fecd
--- /dev/null
+++ b/eventmesh-connector-plugin/eventmesh-connector-api/src/main/java/org/apache/eventmesh/api/connector/storage/StorageConnectorInfo.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.api.connector.storage;
+
+import lombok.Data;
+
+@Data
+public class StorageConnectorInfo {
+
+	private boolean distinguishTopic;
+}
diff --git a/eventmesh-connector-plugin/eventmesh-connector-api/src/main/java/org/apache/eventmesh/api/connector/storage/StorageConnectorProxy.java b/eventmesh-connector-plugin/eventmesh-connector-api/src/main/java/org/apache/eventmesh/api/connector/storage/StorageConnectorProxy.java
new file mode 100644
index 00000000..a0339c65
--- /dev/null
+++ b/eventmesh-connector-plugin/eventmesh-connector-api/src/main/java/org/apache/eventmesh/api/connector/storage/StorageConnectorProxy.java
@@ -0,0 +1,164 @@
+/*
+ * 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.api.connector.storage;
+
+import org.apache.eventmesh.api.AbstractContext;
+import org.apache.eventmesh.api.RequestReplyCallback;
+import org.apache.eventmesh.api.SendCallback;
+import org.apache.eventmesh.api.connector.storage.data.PullRequest;
+import org.apache.eventmesh.api.connector.storage.data.TopicInfo;
+import org.apache.eventmesh.api.connector.storage.metadata.RouteHandler;
+import org.apache.eventmesh.api.connector.storage.metadata.StorageMetaServcie;
+import org.apache.eventmesh.api.connector.storage.reply.ReplyOperationService;
+import org.apache.eventmesh.api.connector.storage.reply.RequestReplyInfo;
+import org.apache.eventmesh.api.exception.ConnectorRuntimeException;
+import org.apache.eventmesh.api.exception.OnExceptionContext;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executor;
+
+import io.cloudevents.CloudEvent;
+
+public class StorageConnectorProxy implements StorageConnector {
+
+    private Map<StorageConnector, String> storageConnectorMap = new ConcurrentHashMap<>();
+
+    private Map<String, StorageConnector> storageConnectorByKeyMap = new ConcurrentHashMap<>();
+
+    private RouteHandler routeHandler = new RouteHandler();
+
+    private ReplyOperationService replyService;
+
+    private StorageMetaServcie storageMetaServcie;
+
+    private Executor executor;
+
+    @Override
+    public void start() {
+    }
+
+    @Override
+    public void shutdown() {
+    }
+
+    @Override
+    public void init(Properties properties) throws Exception {
+    }
+
+    public void setConnector(StorageConnector storageConnector, String key) {
+        storageConnectorMap.put(storageConnector, key);
+        storageConnectorByKeyMap.put(key, storageConnector);
+    }
+
+    @Override
+    public void publish(CloudEvent cloudEvent, SendCallback sendCallback) throws Exception {
+        executor.execute(new Runnable() {
+            @Override
+            public void run() {
+                StorageConnectorProxy.this.doPublish(cloudEvent, sendCallback);
+            }
+        });
+    }
+
+    private void doPublish(CloudEvent cloudEvent, SendCallback sendCallback) {
+        try {
+            StorageConnector storageConnector = routeHandler.select();
+            if (storageConnector instanceof StorageConnectorMetedata
+                && !storageMetaServcie.isTopic(storageConnector, CloudEventUtils.getTopic(cloudEvent))) {
+                TopicInfo topicInfo = new TopicInfo();
+                StorageConnectorMetedata storageConnectorMetedata = (StorageConnectorMetedata) storageConnector;
+                storageConnectorMetedata.createTopic(topicInfo);
+            }
+            storageConnector.publish(cloudEvent, sendCallback);
+        } catch (Exception e) {
+            sendCallback.onException(createOnExceptionContext(e, cloudEvent));
+        }
+    }
+
+    private OnExceptionContext createOnExceptionContext(Exception e, CloudEvent cloudEvent) {
+        OnExceptionContext onExceptionContext = new OnExceptionContext();
+        onExceptionContext.setException(new ConnectorRuntimeException(e));
+        return onExceptionContext;
+    }
+
+    @Override
+    public void request(CloudEvent cloudEvent, RequestReplyCallback requestReplyCallback, long timeout)
+        throws Exception {
+        executor.execute(new Runnable() {
+            @Override
+            public void run() {
+                StorageConnectorProxy.this.doRequest(cloudEvent, requestReplyCallback, timeout);
+            }
+        });
+
+    }
+
+    public void doRequest(CloudEvent cloudEvent, RequestReplyCallback requestReplyCallback, long timeout) {
+        try {
+            StorageConnector storageConnector = routeHandler.select();
+            String key = storageConnectorMap.get(storageConnector);
+            CloudEventUtils.setValue(cloudEvent, "nodeAddress", key);
+            storageConnector.request(cloudEvent, requestReplyCallback, timeout);
+            Long storageId = (Long) cloudEvent.getExtension("storageId");
+            RequestReplyInfo requestReplyInfo = new RequestReplyInfo();
+            requestReplyInfo.setStorageId(storageId);
+            requestReplyInfo.setTimeOut(System.currentTimeMillis() + timeout);
+            requestReplyInfo.setRequestReplyCallback(requestReplyCallback);
+            replyService.setRequestReplyInfo(null, cloudEvent.getType(), storageId, requestReplyInfo);
+        } catch (Exception e) {
+            requestReplyCallback.onException(e);
+        }
+    }
+
+    @Override
+    public void updateOffset(List<CloudEvent> cloudEvents, AbstractContext context) {
+        executor.execute(new Runnable() {
+            @Override
+            public void run() {
+                StorageConnectorProxy.this.doUpdateOffset(cloudEvents, context);
+            }
+        });
+    }
+
+    private void doUpdateOffset(List<CloudEvent> cloudEvents, AbstractContext context) {
+        for (CloudEvent cloudEvent : cloudEvents) {
+            try {
+                StorageConnector storageConnector = storageConnectorByKeyMap
+                    .get(CloudEventUtils.getNodeAdress(cloudEvent));
+                storageConnector.updateOffset(cloudEvents, context);
+            } catch (Exception e) {
+
+            }
+        }
+    }
+
+    @Override
+    public List<CloudEvent> pull(PullRequest pullRequest) {
+        return null;
+    }
+
+    @Override
+    public boolean reply(CloudEvent cloudEvent, SendCallback sendCallback) throws Exception {
+        StorageConnector storageConnector = storageConnectorByKeyMap.get(CloudEventUtils.getNodeAdress(cloudEvent));
+        return storageConnector.reply(cloudEvent, sendCallback);
+    }
+
+}
diff --git a/eventmesh-connector-plugin/eventmesh-connector-api/src/main/java/org/apache/eventmesh/api/connector/storage/StorageConnectorService.java b/eventmesh-connector-plugin/eventmesh-connector-api/src/main/java/org/apache/eventmesh/api/connector/storage/StorageConnectorService.java
new file mode 100644
index 00000000..287c2b91
--- /dev/null
+++ b/eventmesh-connector-plugin/eventmesh-connector-api/src/main/java/org/apache/eventmesh/api/connector/storage/StorageConnectorService.java
@@ -0,0 +1,163 @@
+/*
+ * 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.api.connector.storage;
+
+import org.apache.eventmesh.api.LifeCycle;
+import org.apache.eventmesh.api.connector.storage.data.PullRequest;
+import org.apache.eventmesh.api.connector.storage.metadata.StorageMetaServcie;
+import org.apache.eventmesh.api.connector.storage.pull.StoragePullService;
+import org.apache.eventmesh.api.connector.storage.reply.ReplyOperationService;
+import org.apache.eventmesh.spi.EventMeshExtensionFactory;
+
+import java.net.URL;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import lombok.Getter;
+
+public class StorageConnectorService implements LifeCycle {
+
+    private static final StorageConnectorService instance = new StorageConnectorService();
+
+    private StoragePullService pullService = new StoragePullService();
+
+    private StorageMetaServcie storageMetaServcie = new StorageMetaServcie();
+
+    private ReplyOperationService replyService = new ReplyOperationService();
+
+    private Map<String, StorageConnector> storageConnectorMap = new HashMap<>();
+
+    private Executor executor;
+
+    private ScheduledExecutorService scheduledExecutor;
+
+    @Getter
+    private StorageConnector storageConnector = new StorageConnectorProxy();
+
+    public static StorageConnectorService getInstance() {
+        return instance;
+    }
+
+    private StorageConnectorService() {
+        this.executor = new ThreadPoolExecutor(Runtime.getRuntime().availableProcessors() * 10,
+            Runtime.getRuntime().availableProcessors() * 100, 1000 * 60 * 60, TimeUnit.SECONDS,
+            new LinkedBlockingQueue<>(), new ThreadFactory() {
+            AtomicInteger index = new AtomicInteger();
+
+            @Override
+            public Thread newThread(Runnable r) {
+                return new Thread(r, "storage-connent-" + index.getAndIncrement());
+            }
+        });
+        this.scheduledExecutor = Executors.newScheduledThreadPool(Runtime.getRuntime().availableProcessors() * 10,
+            new ThreadFactory() {
+                AtomicInteger index = new AtomicInteger();
+
+                @Override
+                public Thread newThread(Runnable r) {
+                    return new Thread(r, "storage-connent-shceduled-" + index.getAndIncrement());
+                }
+            });
+        this.storageMetaServcie = new StorageMetaServcie();
+        this.storageMetaServcie.setScheduledExecutor(scheduledExecutor);
+        this.storageMetaServcie.setStoragePullService(pullService);
+        this.replyService.setExecutor(executor);
+        this.pullService.setExecutor(executor);
+        this.executor.execute(pullService);
+        this.scheduled();
+
+    }
+
+    public void scheduled() {
+        scheduledExecutor.scheduleWithFixedDelay(new Runnable() {
+            @Override
+            public void run() {
+                storageMetaServcie.pullMeteData();
+            }
+        }, 5, 1000, TimeUnit.MILLISECONDS);
+        scheduledExecutor.scheduleWithFixedDelay(new Runnable() {
+            @Override
+            public void run() {
+                replyService.execute();
+            }
+        }, 5, 5, TimeUnit.MILLISECONDS);
+    }
+
+    public StorageConnector createConsumerByStorageConnector(Properties properties) {
+        StorageConnector storageConnector = this.createConsumerByStorageConnector(properties);
+        if (storageConnector instanceof StorageConnectorMetedata) {
+            this.storageMetaServcie.registerStorageConnector((StorageConnectorMetedata) storageConnector);
+        }
+
+        return storageConnector;
+    }
+
+    public StorageConnector createProducerByStorageConnector(Properties properties, List<PullRequest> pullRequests) {
+        StorageConnector storageConnector = this.createConsumerByStorageConnector(properties);
+        this.storageMetaServcie.registerPullRequest(pullRequests, storageConnector);
+        return storageConnector;
+    }
+
+    public StorageConnector createStorageConnector(Properties properties) throws Exception {
+        URL url = new URL(properties.getProperty(""));
+        String host = url.getHost() + ":" + url.getPort();
+        String[] hosts = host.split(",");
+        StorageConnectorProxy connectorProxy = new StorageConnectorProxy();
+        for (String address : hosts) {
+            StorageConnector storageConnector = EventMeshExtensionFactory.getExtension(StorageConnector.class,
+                url.getProtocol());
+            properties.setProperty("nodeAddress", address);
+            properties.setProperty("protocol", url.getProtocol());
+            storageConnector.init(properties);
+            String key = url.getProtocol() + "://" + address;
+            connectorProxy.setConnector(storageConnector, key);
+            storageConnectorMap.put(key, storageConnector);
+        }
+
+        return connectorProxy;
+    }
+
+    @Override
+    public boolean isStarted() {
+        return true;
+    }
+
+    @Override
+    public boolean isClosed() {
+        return false;
+    }
+
+    @Override
+    public void start() {
+    }
+
+    @Override
+    public void shutdown() {
+        storageConnectorMap.values().forEach(value -> value.shutdown());
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@eventmesh.apache.org
For additional commands, e-mail: commits-help@eventmesh.apache.org