diff --git a/pom.xml b/pom.xml index 05a6583b..ffb49418 100644 --- a/pom.xml +++ b/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 @@ -31,56 +33,6 @@ pom https://rocketmq.apache.org/ - - - git@github.com:apache/rocketmq-streams.git - scm:git:git@github.com:apache/rocketmq-streams.git - scm:git:git@github.com:apache/rocketmq-streams.git - 1.0.0-preview - - - - - Development List - dev-subscribe@rocketmq.apache.org - dev-unsubscribe@rocketmq.apache.org - dev@rocketmq.apache.org - - - User List - users-subscribe@rocketmq.apache.org - users-unsubscribe@rocketmq.apache.org - users@rocketmq.apache.org - - - Commits List - commits-subscribe@rocketmq.apache.org - commits-unsubscribe@rocketmq.apache.org - commits@rocketmq.apache.org - - - - - - Apache RocketMQ - Apache RocketMQ of ASF - https://rocketmq.apache.org/ - - - - - Apache Software Foundation - http://www.apache.org - - - - - Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 - repo - - - rocketmq-streams-commons rocketmq-streams-transport-minio @@ -112,13 +64,13 @@ 1.8 UTF-8 ${java.encoding} - 1.2.17 1.1 - 3.2.13.RELEASE + 5.1.14.RELEASE 1.0-rc5 5.1.40 1.2.83 2.3.2 + 4.5.13 2.7 4.13.1 @@ -140,8 +92,10 @@ 2.5 7.4.0 1.1.0 - true 1.2.2 + 5.3.0 + 2.56 + 1.7.36 @@ -167,10 +121,7 @@ build_without_test.sh NOTICE LICENSE - README.md - README-chinese.md - QUICKSTART.md - quick_start.md + *.md .github/** */target/** */*.iml @@ -202,18 +153,6 @@ versions-maven-plugin 2.2 - - org.codehaus.mojo - cobertura-maven-plugin - 2.7 - - - html - xml - - - - org.jacoco jacoco-maven-plugin @@ -260,11 +199,6 @@ rocketmq-streams-commons ${project.version} - - org.apache.rocketmq - rocketmq-streams-channel-es - ${project.version} - org.apache.rocketmq rocketmq-streams-channel-mqtt @@ -305,11 +239,6 @@ rocketmq-streams-db-operator ${project.version} - - org.apache.rocketmq - rocketmq-streams-dim - ${project.version} - org.apache.rocketmq rocketmq-streams-filter @@ -331,11 +260,6 @@ - - org.apache.rocketmq - rocketmq-streams-script-python - ${project.version} - org.apache.rocketmq rocketmq-streams-serviceloader @@ -366,7 +290,6 @@ rocketmq-streams-channel-rocketmq ${project.version} - org.apache.rocketmq rocketmq-streams-examples @@ -398,12 +321,6 @@ ${rocketmq.version} - - org.apache.rocketmq - rocketmq-tools - ${rocketmq.version} - - @@ -437,9 +354,9 @@ - log4j - log4j - ${log4j.version} + org.slf4j + slf4j-log4j12 + ${slf4j-log4j12.version} @@ -586,6 +503,12 @@ elasticsearch-rest-client ${elasticsearch.version} + + + org.elasticsearch + elasticsearch + ${elasticsearch.version} + org.apache.kafka kafka_2.12 @@ -598,6 +521,17 @@ org.eclipse.paho.client.mqttv3 ${paho.version} + + + de.ruedigermoeller + fst + ${fst.version} + + + com.esotericsoftware + kryo + ${kryo.version} + diff --git a/rocketmq-streams-channel-db/pom.xml b/rocketmq-streams-channel-db/pom.xml index fb5caa04..82a66995 100755 --- a/rocketmq-streams-channel-db/pom.xml +++ b/rocketmq-streams-channel-db/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.rocketmq diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/AbstractMultiTableSink.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/AbstractMultiTableSink.java index a46bb16b..ef9380dd 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/AbstractMultiTableSink.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/AbstractMultiTableSink.java @@ -34,7 +34,7 @@ public abstract class AbstractMultiTableSink extends EnhanceDBSink { protected transient AtomicLong messageCount = new AtomicLong(0); protected transient MultiTableSplitFunction multiTableSplitFunction; - public AbstractMultiTableSink(){ + public AbstractMultiTableSink() { } public AbstractMultiTableSink(String url, String userName, String password) { @@ -44,16 +44,16 @@ public AbstractMultiTableSink(String url, String userName, String password) { } @Override - protected boolean initConfigurable(){ + protected boolean initConfigurable() { Iterator it = tableSinks.values().iterator(); - while(it.hasNext()){ + while (it.hasNext()) { it.next().initConfigurable(); } return true; } @Override - public boolean batchAdd(IMessage message, ISplit split) { + public boolean batchAdd(IMessage message, ISplit split) { EnhanceDBSink sink = getOrCreateDBSink(split.getQueueId()); boolean success = sink.batchAdd(message, split); @@ -68,7 +68,7 @@ public boolean batchAdd(IMessage message, ISplit split) { @Override public boolean batchAdd(IMessage message) { - ISplit split = getSplitFromMessage(message); + ISplit split = getSplitFromMessage(message); return batchAdd(message, split); } @@ -142,7 +142,7 @@ protected EnhanceDBSink getOrCreateDBSink(String splitId) { protected abstract String createTableName(String splitId); - protected abstract ISplit getSplitFromMessage(IMessage message); + protected abstract ISplit getSplitFromMessage(IMessage message); protected class SingleDBSinkCache extends MessageCache { diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DynamicMultipleDBSink.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DynamicMultipleDBSink.java index ebcf9d8b..d57fe015 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DynamicMultipleDBSink.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DynamicMultipleDBSink.java @@ -32,7 +32,7 @@ public class DynamicMultipleDBSink extends AbstractMultiTableSink implements IAf String logicTableName; String fieldName; - public DynamicMultipleDBSink(){ + public DynamicMultipleDBSink() { } public String getLogicTableName() { @@ -63,21 +63,21 @@ protected String createTableName(String splitId) { } @Override - protected ISplit getSplitFromMessage(IMessage message) { + protected ISplit getSplitFromMessage(IMessage message) { return this.multiTableSplitFunction.createSplit(message); } - @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - if(this.multiTableSplitFunction == null){ + if (this.multiTableSplitFunction == null) { this.multiTableSplitFunction = new MultiTableSplitFunction() { @Override - public ISplit createSplit(IMessage message) { + public ISplit createSplit(IMessage message) { return new DynamicMultipleDBSplit(message.getMessageBody().getString(fieldName), logicTableName); } + @Override public String createTableFromSplitId(String splitId) { return splitId; @@ -86,6 +86,5 @@ public String createTableFromSplitId(String splitId) { } - } } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SelfMultiTableSink.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SelfMultiTableSink.java index dc5271ad..709fb77b 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SelfMultiTableSink.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SelfMultiTableSink.java @@ -41,7 +41,7 @@ protected String createTableName(String splitId) { } @Override - protected ISplit getSplitFromMessage(IMessage message) { + protected ISplit getSplitFromMessage(IMessage message) { return multiTableSplitFunction.createSplit(message); } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SplitBySerialNumber.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SplitBySerialNumber.java index c2a49b75..5c3ca3eb 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SplitBySerialNumber.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SplitBySerialNumber.java @@ -30,7 +30,7 @@ protected String createTableName(String splitId) { } @Override - protected ISplit getSplitFromMessage(IMessage message) { + protected ISplit getSplitFromMessage(IMessage message) { return null; } } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SplitByTimeMultiTableSink.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SplitByTimeMultiTableSink.java index 87a2b3e7..d40c59f4 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SplitByTimeMultiTableSink.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/SplitByTimeMultiTableSink.java @@ -30,7 +30,7 @@ protected String createTableName(String splitId) { } @Override - protected ISplit getSplitFromMessage(IMessage message) { + protected ISplit getSplitFromMessage(IMessage message) { return null; } } diff --git a/rocketmq-streams-channel-es/pom.xml b/rocketmq-streams-channel-es/pom.xml index 15723b3c..b4ce4d85 100755 --- a/rocketmq-streams-channel-es/pom.xml +++ b/rocketmq-streams-channel-es/pom.xml @@ -1,27 +1,14 @@ - - + 4.0.0 org.apache.rocketmq rocketmq-streams 1.0.2-preview-SNAPSHOT - + rocketmq-streams-channel-es ROCKETMQ STREAMS :: channel-es jar @@ -39,6 +26,12 @@ org.apache.rocketmq rocketmq-streams-configurable + + + org.elasticsearch + elasticsearch + + diff --git a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESChannelBuilder.java b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESChannelBuilder.java new file mode 100644 index 00000000..b85a48f3 --- /dev/null +++ b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESChannelBuilder.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.rocketmq.streams.es.sink; + +import com.alibaba.fastjson.JSONObject; +import com.google.auto.service.AutoService; +import java.util.Properties; +import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; +import org.apache.rocketmq.streams.common.channel.builder.IShuffleChannelBuilder; +import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.model.ServiceName; +import org.apache.rocketmq.streams.common.utils.ConfigurableUtil; + +@AutoService(IChannelBuilder.class) +@ServiceName(value = ESChannelBuilder.TYPE, aliasName = "ES") +public class ESChannelBuilder implements IChannelBuilder { + public static final String TYPE = "es"; + + protected JSONObject createFormatProperty(Properties properties) { + JSONObject formatProperties = new JSONObject(); + for (Object object : properties.keySet()) { + String key = (String) object; + if ("type".equals(key)) { + continue; + } + formatProperties.put(key, properties.get(key)); + } + IChannelBuilder.formatPropertiesName(formatProperties, properties, "host", "endPoint"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "esIndex", "index"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "esIndexType", "typeName");; + IChannelBuilder.formatPropertiesName(formatProperties, properties, "esMsgId", "es_msg_id");; + return formatProperties; + } + + @Override public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + throw new RuntimeException("can not support source for ES"); + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + return (ISink) ConfigurableUtil.create(ESSinkOnlyChannel.class.getName(), namespace, name, createFormatProperty(properties), null); + } + +} diff --git a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkBuilder.java b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkBuilder.java index 2a28eb9e..9ebd822b 100644 --- a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkBuilder.java +++ b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkBuilder.java @@ -70,6 +70,7 @@ protected JSONObject createFormatProperty(Properties properties) { IChannelBuilder.formatPropertiesName(formatProperties, properties, "esIndexType", "esindextype"); IChannelBuilder.formatPropertiesName(formatProperties, properties, "host", "endpoint"); IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "maxthread"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "esMsgId", "es_msg_id"); return formatProperties; } diff --git a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkOnlyChannel.java b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkOnlyChannel.java index d5c37b34..c14063ab 100644 --- a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkOnlyChannel.java +++ b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/ESSinkOnlyChannel.java @@ -16,6 +16,8 @@ */ package org.apache.rocketmq.streams.es.sink; +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.JSONObject; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -27,9 +29,11 @@ import org.apache.http.auth.UsernamePasswordCredentials; import org.apache.http.client.CredentialsProvider; import org.apache.http.impl.client.BasicCredentialsProvider; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.utils.StringUtil; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; @@ -53,11 +57,11 @@ public class ESSinkOnlyChannel extends AbstractSink { private String authUsername; @ENVDependence private String authPassword; - private int socketTimeOut = 1000; + private int socketTimeOut = 5 * 60 * 1000;; - private int connectTimeOut = 10000; + private int connectTimeOut = 5 * 60 * 1000;; - private int connectionRequestTimeOut = 500; + private int connectionRequestTimeOut = 5 * 60 * 1000;; private String schema = "http"; @ENVDependence @@ -65,6 +69,8 @@ public class ESSinkOnlyChannel extends AbstractSink { private String esIndexType = "log"; + protected String esMsgId; + private transient RestHighLevelClient client; public ESSinkOnlyChannel() { @@ -97,6 +103,7 @@ public void setAuthPassword(String authPassword) { @Override protected boolean initConfigurable() { + super.initConfigurable(); System.setProperty("es.set.netty.runtime.available.processors", "false"); if (client == null) { RestClientBuilder builder = RestClient.builder(new HttpHost(host, Integer.parseInt(port), schema)); @@ -123,13 +130,35 @@ protected boolean initConfigurable() { return true; } + private List generateRequests(List messages) { List requests = new ArrayList<>(); messages.forEach(message -> { IndexRequest indexRequest = new IndexRequest(esIndex); Object object = message.getMessageValue(); + if(object!=null&&!(object instanceof Map)){ + String str=object.toString(); + if(str.startsWith("{")&&str.endsWith("}")){ + try { + JSONObject jsonObject= JSON.parseObject(str); + object=jsonObject; + }catch (Exception e){ + LOG.warn("the sink msg is not json, convert error"); + } + + } + } if (object instanceof Map) { indexRequest.source((Map) object); + if(StringUtil.isNotEmpty(esMsgId)){ + Map map=(Map)object; + Object msgId=map.get(esMsgId); + if(msgId!=null){ + indexRequest.id(msgId.toString()); + } + } + + } else { indexRequest.source(object.toString()); } @@ -148,6 +177,7 @@ public boolean batchInsert(List messages) { try { response = client.bulk(bulkRequest, RequestOptions.DEFAULT); } catch (IOException e) { + e.printStackTrace(); LOG.error("batch insert message to es exception " + e); return false; } @@ -234,4 +264,11 @@ public void setPort(String port) { this.port = port; } + public String getEsMsgId() { + return esMsgId; + } + + public void setEsMsgId(String esMsgId) { + this.esMsgId = esMsgId; + } } diff --git a/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/EsClient.java b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/EsClient.java new file mode 100644 index 00000000..ce800909 --- /dev/null +++ b/rocketmq-streams-channel-es/src/main/java/org/apache/rocketmq/streams/es/sink/EsClient.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.es.sink; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; + +/** + * @author zhangliang + *

+ * 2021-04-16 17:20 + */ +public class EsClient { + + private static final Log logger = LogFactory.getLog(EsClient.class); + + //private static final Logger logger = LogManager.getLogger(EsClient.class); + + private RestHighLevelClient esClient; + + private Object object = new Object(); + + private static final int socketTimeOut = 5 * 60 * 1000; + + private static final int connectTimeOut = 5 * 60 * 1000; + + private static final int connectionRequestTimeOut = 5 * 60 * 1000; + + private static final boolean needAuth = true; + + private String host; + + private String port; + + private String username; + + private String password; + + private static String SCHEME = "http"; + + public EsClient(String host, String port, String username, String password) { + this.host = host; + this.port = port; + this.username = username; + this.password = password; + } + + public void init() { + getEsClient(); + } + + public RestHighLevelClient getEsClient() { + if (esClient == null) { + synchronized (object) { + if (esClient == null) { + esClient = createClient(); + } + } + } + return esClient; + } + + private RestHighLevelClient createClient() { + logger.info("esClient createClient host=" + host + " port=" + port); + try { + //解决netty冲突问题 + System.setProperty("es.set.netty.runtime.available.processors", "false"); + RestClientBuilder builder = RestClient.builder(new HttpHost(host, Integer.parseInt(port), SCHEME)); + builder.setRequestConfigCallback((config) -> { + config.setConnectionRequestTimeout(connectionRequestTimeOut); + config.setConnectTimeout(connectTimeOut); + config.setSocketTimeout(socketTimeOut); + return config; + }); + if (needAuth) { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials(AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + builder.setHttpClientConfigCallback(new RestClientBuilder.HttpClientConfigCallback() { + @Override + public HttpAsyncClientBuilder customizeHttpClient(HttpAsyncClientBuilder httpClientBuilder) { + return httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + } + }); + } + esClient = new RestHighLevelClient(builder); + } catch (Exception e) { + logger.error("esClient createClient host=" + host + " port=" + port + " error=" + e.getMessage(), e); + throw e; + } + return esClient; + } + + public void rebuildEsCilent() { + closeClient(); + getEsClient(); + } + + private void closeClient() { + logger.error("esClient closeClient host=" + host + " port=" + port); + if (esClient != null) { + try { + esClient.close(); + esClient = null; + logger.error("esClient closeClient success host=" + host + " port=" + port); + } catch (IOException e) { + logger.error("esClient closeClient failed host=" + host + " port=" + port + " error=" + e.getMessage(), e); + } + } + } +} diff --git a/rocketmq-streams-channel-http/pom.xml b/rocketmq-streams-channel-http/pom.xml index df70fe02..ac2cbc17 100644 --- a/rocketmq-streams-channel-http/pom.xml +++ b/rocketmq-streams-channel-http/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + org.apache.rocketmq rocketmq-streams diff --git a/rocketmq-streams-channel-mqtt/pom.xml b/rocketmq-streams-channel-mqtt/pom.xml index 1ebfd008..637f656d 100644 --- a/rocketmq-streams-channel-mqtt/pom.xml +++ b/rocketmq-streams-channel-mqtt/pom.xml @@ -1,21 +1,7 @@ - - + rocketmq-streams org.apache.rocketmq diff --git a/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/source/PahoSource.java b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/source/PahoSource.java index a439f5cc..11490ec3 100644 --- a/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/source/PahoSource.java +++ b/rocketmq-streams-channel-mqtt/src/main/java/org/apache/rocketmq/streams/mqtt/source/PahoSource.java @@ -18,7 +18,9 @@ import com.alibaba.fastjson.JSONObject; import java.nio.charset.StandardCharsets; -import org.apache.rocketmq.streams.common.channel.source.AbstractBatchSource; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.rocketmq.streams.common.channel.source.AbstractSource; +import org.apache.rocketmq.streams.common.utils.RuntimeUtil; import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; import org.eclipse.paho.client.mqttv3.MqttCallback; import org.eclipse.paho.client.mqttv3.MqttClient; @@ -26,8 +28,12 @@ import org.eclipse.paho.client.mqttv3.MqttException; import org.eclipse.paho.client.mqttv3.MqttMessage; import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class PahoSource extends AbstractBatchSource { +public class PahoSource extends AbstractSource { + + private static final Logger LOGGER = LoggerFactory.getLogger(PahoSource.class); private String url; private String clientId; @@ -54,8 +60,7 @@ public PahoSource(String url, String clientId, String topic, String username, St this(url, clientId, topic, username, password, true, 10, 60, true); } - public PahoSource(String url, String clientId, String topic, String username, String password, Boolean cleanSession, - Integer connectionTimeout, Integer aliveInterval, Boolean automaticReconnect) { + public PahoSource(String url, String clientId, String topic, String username, String password, Boolean cleanSession, Integer connectionTimeout, Integer aliveInterval, Boolean automaticReconnect) { this.url = url; this.clientId = clientId; this.topic = topic; @@ -68,14 +73,16 @@ public PahoSource(String url, String clientId, String topic, String username, St } private transient MqttClient client; + protected transient AtomicLong offsetGenerator; @Override protected boolean startSource() { try { this.client = new MqttClient(url, clientId, new MemoryPersistence()); + this.offsetGenerator = new AtomicLong(System.currentTimeMillis()); this.client.setCallback(new MqttCallback() { @Override public void connectionLost(Throwable throwable) { - System.out.println("Reconnecting to broker: " + url); + LOGGER.info("Reconnecting to broker: " + url); while (true) { MqttConnectOptions connOpts = new MqttConnectOptions(); if (username != null && password != null) { @@ -105,13 +112,15 @@ public PahoSource(String url, String clientId, String topic, String username, St } try { - client.connect(connOpts); - System.out.println("Reconnecting success"); + if (!client.isConnected()) { + client.connect(connOpts); + LOGGER.info("Reconnecting success"); + } client.subscribe(topic); break; } catch (MqttException e) { try { - System.err.println("Reconnecting err: " + e.getMessage()); + LOGGER.error("Reconnecting err: " + e.getMessage()); e.printStackTrace(); Thread.sleep(10000); } catch (InterruptedException ex) { @@ -123,11 +132,12 @@ public PahoSource(String url, String clientId, String topic, String username, St @Override public void messageArrived(String s, MqttMessage mqttMessage) throws Exception { JSONObject msg = create(new String(mqttMessage.getPayload(), StandardCharsets.UTF_8)); - doReceiveMessage(msg, false); + msg.put("__topic", s); + doReceiveMessage(msg, false, RuntimeUtil.getDipperInstanceId(), offsetGenerator.incrementAndGet() + ""); } @Override public void deliveryComplete(IMqttDeliveryToken token) { - System.out.println("deliveryComplete---------" + token.isComplete()); + LOGGER.info("deliveryComplete---------" + token.isComplete()); } }); @@ -158,9 +168,11 @@ public PahoSource(String url, String clientId, String topic, String username, St connOpts.setAutomaticReconnect(this.automaticReconnect); } - System.out.println("Connecting to broker: " + url); - this.client.connect(connOpts); - System.out.println("Connected"); + LOGGER.info("Connecting to broker: " + url); + if (!this.client.isConnected()) { + this.client.connect(connOpts); + LOGGER.info("Connected"); + } this.client.subscribe(topic); return true; } catch (MqttException e) { @@ -172,10 +184,11 @@ public PahoSource(String url, String clientId, String topic, String username, St @Override public void destroy() { super.destroy(); try { - if (this.client != null) { + if (this.client != null && this.client.isConnected()) { this.client.disconnect(); this.client.close(); } + super.destroy(); } catch (MqttException e) { e.printStackTrace(); } diff --git a/rocketmq-streams-channel-rocketmq/pom.xml b/rocketmq-streams-channel-rocketmq/pom.xml index c10f3e9e..31f6d478 100644 --- a/rocketmq-streams-channel-rocketmq/pom.xml +++ b/rocketmq-streams-channel-rocketmq/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + rocketmq-streams org.apache.rocketmq @@ -31,6 +33,10 @@ org.apache.rocketmq rocketmq-streams-commons + + org.apache.rocketmq + rocketmq-streams-serviceloader + org.apache.rocketmq @@ -60,26 +66,6 @@ test - - org.slf4j - slf4j-api - 1.7.26 - - - org.slf4j - jcl-over-slf4j - 1.7.26 - - - org.slf4j - slf4j-log4j12 - 1.7.26 - - - log4j - log4j - 1.2.17 - \ No newline at end of file diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/debug/DebugWriter.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/debug/DebugWriter.java index 2726dd8e..1c622a81 100644 --- a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/debug/DebugWriter.java +++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/debug/DebugWriter.java @@ -28,90 +28,98 @@ import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.common.utils.FileUtil; import org.apache.rocketmq.streams.queue.RocketMQMessageQueue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class DebugWriter { + private static final Logger LOGGER = LoggerFactory.getLogger(DebugWriter.class); - protected String dir="/tmp/rocksmq-streams/mq"; - protected static Map debugWriterMap=new HashMap<>(); - public static DebugWriter getInstance(String topic){ - DebugWriter debugWriter=debugWriterMap.get(topic); - if(debugWriter==null){ - debugWriter=new DebugWriter(); - debugWriterMap.put(topic,debugWriter); + protected String dir = "/tmp/rocksmq-streams/mq"; + protected static Map debugWriterMap = new HashMap<>(); + + public static DebugWriter getInstance(String topic) { + DebugWriter debugWriter = debugWriterMap.get(topic); + if (debugWriter == null) { + debugWriter = new DebugWriter(); + debugWriterMap.put(topic, debugWriter); } return debugWriter; } - public static boolean isOpenDebug(){ + public static boolean isOpenDebug() { return false; } - public DebugWriter(){} - public DebugWriter(String dir){ - this.dir=dir; + public DebugWriter() { + } + + public DebugWriter(String dir) { + this.dir = dir; } /** * write offset 2 file + * * @param offsets */ - public void writeSaveOffset(Map offsets){ - if(isOpenDebug()==false){ + public void writeSaveOffset(Map offsets) { + if (!isOpenDebug()) { return; } - String path=dir+"/offsets/offset.txt"; - if(offsets==null||offsets.size()==0){ + String path = dir + "/offsets/offset.txt"; + if (offsets == null || offsets.size() == 0) { return; } Iterator> it = offsets.entrySet().iterator(); - List rows=new ArrayList<>(); - while(it.hasNext()){ - Map.Entry entry=it.next(); - String queueId=new RocketMQMessageQueue(entry.getKey()).getQueueId(); - JSONObject msg=new JSONObject(); - Long offset=entry.getValue().get(); - msg.put(queueId,offset); + List rows = new ArrayList<>(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + String queueId = new RocketMQMessageQueue(entry.getKey()).getQueueId(); + JSONObject msg = new JSONObject(); + Long offset = entry.getValue().get(); + msg.put(queueId, offset); msg.put("saveTime", DateUtil.getCurrentTimeString()); msg.put("queueId", queueId); rows.add(msg.toJSONString()); } - FileUtil.write(path,rows,true); + FileUtil.write(path, rows, true); } - public void writeSaveOffset(MessageQueue messageQueue, AtomicLong offset){ - if(isOpenDebug()==false){ + public void writeSaveOffset(MessageQueue messageQueue, AtomicLong offset) { + if (!isOpenDebug()) { return; } - Map offsets=new HashMap<>(); - offsets.put(messageQueue,offset); + Map offsets = new HashMap<>(); + offsets.put(messageQueue, offset); writeSaveOffset(offsets); } - - public void receiveFirstData(String queueId,Long offset){ - if(isOpenDebug()==false){ + public void receiveFirstData(String queueId, Long offset) { + if (!isOpenDebug()) { return; } - Map offsets=load(); - Long saveOffset=offsets.get(queueId); - System.out.println("queueId is "+queueId+"current offset "+offset+"===="+saveOffset); + Map offsets = load(); + Long saveOffset = offsets.get(queueId); + LOGGER.info("queueId is " + queueId + "current offset " + offset + "====" + saveOffset); } + /** * load offsets + * * @return */ - public Map load(){ - if(isOpenDebug()==false){ + public Map load() { + if (!isOpenDebug()) { return null; } - String path=dir+"/offsets/offset.txt"; - List lines=FileUtil.loadFileLine(path); - Map offsets=new HashMap<>(); - for(String line:lines){ - JSONObject row=JSONObject.parseObject(line); - String queueId=row.getString("queueId"); - offsets.put(queueId,row.getLong(queueId)); + String path = dir + "/offsets/offset.txt"; + List lines = FileUtil.loadFileLine(path); + Map offsets = new HashMap<>(); + for (String line : lines) { + JSONObject row = JSONObject.parseObject(line); + String queueId = row.getString("queueId"); + offsets.put(queueId, row.getLong(queueId)); } return offsets; } diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java index e693bf56..a8998dad 100644 --- a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java +++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -104,7 +105,7 @@ protected boolean batchInsert(List messages) { Map messageQueueMap = new HashMap<>();//if has queue id in message, save the map for queueid 2 messagequeeue String defaultQueueId = "";//message is not contains queue ,use default for (IMessage msg : messages) { - ISplit channelQueue = getSplit(msg); + ISplit channelQueue = (ISplit) getSplit(msg); String queueId = defaultQueueId; if (channelQueue != null) { queueId = channelQueue.getQueueId(); @@ -151,10 +152,6 @@ protected void initProducer() { destroy(); producer = new DefaultMQProducer(null, groupName + "producer", rpcHook,false, null); try { - //please not use the code,the name srv addr may be empty in jmenv -// if (this.namesrvAddr == null || "".equals(this.namesrvAddr)) { -// throw new RuntimeException("namesrvAddr can not be null."); -// } if (StringUtil.isNotEmpty(this.namesrvAddr)) { producer.setNamesrvAddr(this.namesrvAddr); @@ -242,19 +239,19 @@ protected void createTopicIfNotExist(int splitNum) { } @Override - public List getSplitList() { + public List> getSplitList() { initProducer(); - List messageQueues = new ArrayList<>(); + List> messageQueues = new ArrayList<>(); try { List messageQueueSet = producer.fetchPublishMessageQueues(topic); - List queueList = new ArrayList<>(); + List> queueList = new ArrayList<>(); for (MessageQueue queue : messageQueueSet) { RocketMQMessageQueue rocketMQMessageQueue = new RocketMQMessageQueue(queue); queueList.add(rocketMQMessageQueue); } - Collections.sort(queueList); + queueList.sort((Comparator) Comparable::compareTo); messageQueues = queueList; } catch (MQClientException e) { return messageQueues; @@ -265,7 +262,7 @@ public List getSplitList() { @Override public int getSplitNum() { - List splits = getSplitList(); + List> splits = getSplitList(); if (splits == null || splits.size() == 0) { return 0; } diff --git a/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/RocketMQChannelTest.java b/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/RocketMQChannelTest.java index fbea1798..cc62a4bf 100644 --- a/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/RocketMQChannelTest.java +++ b/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/RocketMQChannelTest.java @@ -66,7 +66,7 @@ public void testSink() { @Test public void testGetSplit() { AbstractSupportShuffleSink sink = (AbstractSupportShuffleSink)createSink(); - List splits = sink.getSplitList(); + List> splits = sink.getSplitList(); System.out.println(splits.size()); } diff --git a/rocketmq-streams-channel-syslog/pom.xml b/rocketmq-streams-channel-syslog/pom.xml index 0d3b6f1a..61e9aac6 100755 --- a/rocketmq-streams-channel-syslog/pom.xml +++ b/rocketmq-streams-channel-syslog/pom.xml @@ -1,21 +1,8 @@ - - + 4.0.0 org.apache.rocketmq diff --git a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannel.java b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannel.java index d771c824..7869244c 100644 --- a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannel.java +++ b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannel.java @@ -53,7 +53,7 @@ public class SyslogChannel extends AbstractChannel implements ISyslogRouter { protected String keywords; protected String ipListStr; - private transient org.graylog2.syslog4j.SyslogIF syslog; + private transient org.graylog2.syslog4j.SyslogIF syslogClient; public SyslogChannel() { } @@ -74,7 +74,7 @@ public SyslogChannel(String serverIp, int port) { @Override protected ISink createSink() { - return new AbstractSink() { + this.sink= new AbstractSink() { @Override protected boolean batchInsert(List messages) { if (messages == null || !syslogClientInit) { @@ -95,32 +95,36 @@ protected boolean batchInsert(List messages) { encode = abstractSource.getEncoding(); } message = URLDecoder.decode(msg.getMessageValue().toString(), encode); - syslog.getConfig().setLocalName(IPUtil.getLocalIP()); - syslog.getConfig().setSendLocalTimestamp(true); - syslog.getConfig().setSendLocalName(true);//如果这个值是false,需要确保json数据无空格 - //本机测试必须设置,否则ip地址变成了127.0.0.1,如果是远端server,必须注释掉这一行,否则server发生覆盖 - //syslog.getConfig().setHost(IPUtil.getLocalIP()); + syslogClient.getConfig().setLocalName(IPUtil.getLocalIP()); + syslogClient.getConfig().setSendLocalTimestamp(true); + syslogClient.getConfig().setSendLocalName(true);//如果这个值是false,需要确保json数据无空格 + if("127.0.0.1".equals(syslogClient.getConfig().getHost())){ + //本机测试必须设置,否则ip地址变成了127.0.0.1,如果是远端server,必须注释掉这一行,否则server发生覆盖 + syslogClient.getConfig().setHost(IPUtil.getLocalIP()); + } } catch (Exception e) { - LOG.error("syslog decode message error " + msg.getMessageValue().toString(), e); + LOG.error("syslogClient decode message error " + msg.getMessageValue().toString(), e); } - syslog.log(level, message); + syslogClient.log(level, message); } - syslog.flush(); + syslogClient.flush(); return true; } }; + return this.sink; } @Override protected ISource createSource() { - return new AbstractUnreliableSource() { + this.source=new AbstractUnreliableSource() { @Override protected boolean startSource() { SyslogChannelManager.start(protol); return true; } }; + return this.source; } @Override @@ -138,13 +142,12 @@ protected boolean initConfigurable() { ipList.add(value); } } - syslog = Syslog.getInstance(protol); - SyslogConfigIF config = syslog.getConfig(); + syslogClient = Syslog.getInstance(protol); + SyslogConfigIF config = syslogClient.getConfig(); config.setHost(serverIp); - config.setPort( - protol == SyslogChannelManager.UDP ? SyslogChannelManager.udpPort : SyslogChannelManager.tcpPort); + config.setPort((protol .equals( SyslogChannelManager.UDP) )? SyslogChannelManager.udpPort : SyslogChannelManager.tcpPort); } catch (Throwable throwable) { - LOG.warn("syslog client init fail " + throwable); + LOG.warn("syslogClient client init fail " + throwable); syslogClientInit = false; } diff --git a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelBuilder.java b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelBuilder.java index 8b1ae928..60ddf528 100644 --- a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelBuilder.java +++ b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelBuilder.java @@ -41,6 +41,8 @@ public class SyslogChannelBuilder implements IChannelBuilder { public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { SyslogChannel source = (SyslogChannel) ConfigurableUtil.create(SyslogChannel.class.getName(), namespace, name, createFormatProperty(properties), null); + source.createSource(); + source.setType(ISource.TYPE); return source; } @@ -52,6 +54,8 @@ public String getType() { @Override public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { SyslogChannel sink = (SyslogChannel) ConfigurableUtil.create(SyslogChannel.class.getName(), namespace, name, createFormatProperty(properties), null); + sink.createSink(); + sink.setType(ISink.TYPE); return sink; } diff --git a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelManager.java b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelManager.java index 19826df5..0e1be82a 100644 --- a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelManager.java +++ b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogChannelManager.java @@ -28,8 +28,8 @@ public class SyslogChannelManager { public static final String TCP_PORT_PROPERTY_KEY = "dipper.syslog.server.tcp.port";//当需要改变端口值时,通过配置文件增加dipper.syslog.server.tcp.port=新端口的值 public static final String UDP_PORT_PROPERTY_KEY = "dipper.syslog.server.udp.port";//当需要改变端口值时,通过配置文件增加dipper.syslog.server.tcp.port=新端口的值 - public transient static int tcpPort = 12345;//syslog server默认端口 - public transient static int udpPort = 12346;//syslog server默认端口 + public static int tcpPort = 12345;//syslog server默认端口 + public static int udpPort = 12346;//syslog server默认端口 private static AtomicBoolean tcpStart = new AtomicBoolean(false);//标记是否启动tcp server,只会启动一次 private static AtomicBoolean updStart = new AtomicBoolean(false);//标记是否启动udp server,只会启动一次 @@ -39,7 +39,7 @@ public class SyslogChannelManager { public static void registeTCP(SyslogChannel syslogRouter) { if (!TCP_CHANNEL.getRouters().contains(syslogRouter)) { TCP_CHANNEL.getRouters().add(syslogRouter); - if (tcpPort == 12345) { + if (tcpPort == 12345&&syslogRouter.getPort()>0) { tcpPort = syslogRouter.getPort(); } } diff --git a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogServer.java b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogServer.java index 4169c433..b7920527 100644 --- a/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogServer.java +++ b/rocketmq-streams-channel-syslog/src/main/java/org/apache/rocketmq/streams/syslog/SyslogServer.java @@ -82,13 +82,16 @@ public class SyslogServer extends AbstractUnreliableSource { SyslogServerSessionEventHandlerIF var5 = new SyslogServerEventHandler(); var3.addEventHandler(var5); setSingleType(true);//单消费者 - return true; + return super.initConfigurable(); } @Override public boolean startSource() { setReceiver(new IStreamOperator() { @Override public Object doMessage(IMessage message, AbstractContext context) { - String hostAddress = message.getMessageBody().getString("_hostAddress"); + String hostAddress = message.getMessageBody().getString("hostAddress"); + if(hostAddress==null){ + return null; + } List syslogChannels = cache.get(hostAddress); LOG.info("receive syslog msg, ip is " + hostAddress + " msg is " + message.getMessageBody()); boolean hasMatch = false; @@ -118,7 +121,7 @@ public class SyslogServer extends AbstractUnreliableSource { return message; } }); - org.graylog2.syslog4j.server.SyslogServer.getThreadedInstance(protocol); + SyslogServerIF serverIF=org.graylog2.syslog4j.server.SyslogServer.getThreadedInstance(protocol); return true; } @@ -194,7 +197,7 @@ public SyslogServerEventHandler() { } @Override public void event(Object var1, SyslogServerIF var2, SocketAddress var3, SyslogServerEventIF var4) { - JSONObject msg = new JSONObject(); + String hostAddress = null; if (InetSocketAddress.class.isInstance(var3)) { InetSocketAddress address = (InetSocketAddress) var3; @@ -230,19 +233,17 @@ public SyslogServerEventHandler() { } } - - msg.put("_facility", var4.getFacility()); - msg.put("_hostName", hostName); - msg.put("_hostAddress", hostAddress); - msg.put("_level", var4.getLevel()); - msg.put("_data", message); - msg.put("_date", DateUtil.format(date)); - msg.put("_tag", tag); - msg.put("_pid", pid); - UserDefinedMessage userDefinedMessage = new UserDefinedMessage(message); - userDefinedMessage.putAll(msg); - userDefinedMessage.put(IMessage.DATA_KEY, message); - doReceiveMessage(userDefinedMessage); + JSONObject msg = new JSONObject(); + msg.put("data",message); + msg.put("facility", var4.getFacility()); + msg.put("hostName", hostName); + msg.put("hostAddress", hostAddress); + msg.put("level", var4.getLevel()); + msg.put("log_time", DateUtil.format(date)); + msg.put("tag", tag); + msg.put("pid", pid); + + doReceiveMessage(msg); } diff --git a/rocketmq-streams-channel-syslog/src/test/java/org/apache/rocketmq/streams/syslog/SyslogClient.java b/rocketmq-streams-channel-syslog/src/test/java/org/apache/rocketmq/streams/syslog/SyslogClient.java index 31b6e5c9..d3c1229c 100644 --- a/rocketmq-streams-channel-syslog/src/test/java/org/apache/rocketmq/streams/syslog/SyslogClient.java +++ b/rocketmq-streams-channel-syslog/src/test/java/org/apache/rocketmq/streams/syslog/SyslogClient.java @@ -36,27 +36,35 @@ public void sendSyslog() throws InterruptedException { syslogChannel.start(new IStreamOperator() { @Override public Object doMessage(IMessage message, AbstractContext context) { - System.out.println(message.getMessageBody()); + if(!message.getHeader().isSystemMessage()){ + System.out.println(message.getMessageBody()); + } return null; } }); - addData(syslogChannel); + System.out.println("start....."); + Thread.sleep(3000); + sendTestData(); Thread.sleep(1000000000l); } - private void addData(IChannel channel) { + + @Test + public void sendTestData() throws InterruptedException { + IChannel channel=createSyslogChannel(); JSONObject msg = new JSONObject(); msg.put("name", "chris"); - //msg.put("host",IPUtil.getLocalIP()); + msg.put("host",IPUtil.getLocalIP()); channel.batchAdd(new Message(msg)); channel.flush(); + Thread.sleep(3000); } private SyslogChannel createSyslogChannel() { - SyslogChannel syslogChannel = new SyslogChannel(); - syslogChannel.setUDPProtol(); + SyslogChannel syslogChannel = new SyslogChannel(IPUtil.getLocalIP(),SyslogChannelManager.tcpPort); + syslogChannel.setTCPProtol(); syslogChannel.addIps(IPUtil.getLocalIP()); - syslogChannel.setServerIp("11.158.144.159"); + System.out.println(IPUtil.getLocalIP()); syslogChannel.init(); return syslogChannel; } diff --git a/rocketmq-streams-clients/pom.xml b/rocketmq-streams-clients/pom.xml index ff0e4cb0..e24e55f3 100644 --- a/rocketmq-streams-clients/pom.xml +++ b/rocketmq-streams-clients/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + rocketmq-streams org.apache.rocketmq @@ -38,6 +40,7 @@ org.apache.rocketmq rocketmq-streams-channel-mqtt + org.apache.rocketmq rocketmq-streams-channel-db @@ -50,6 +53,11 @@ org.apache.rocketmq rocketmq-streams-filter + + org.apache.rocketmq + rocketmq-streams-channel-syslog + + org.apache.rocketmq rocketmq-streams-window diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java index 8ee45681..35668b95 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java @@ -159,6 +159,25 @@ public DataStream fromMqtt(String url, String clientId, String topic, String use return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); } +// public DataStream fromKafka(String endpoint, String topic, String groupName) { +// return fromKafka(endpoint, topic, groupName, true); +// } +// +// public DataStream fromKafka(String endpoint, String topic, String groupName, Boolean isJson) { +// return fromKafka(endpoint, topic, groupName, isJson, 1); +// } +// +// public DataStream fromKafka(String endpoint, String topic, String groupName, Boolean isJson, int maxThread) { +// KafkaSource kafkaChannel = new KafkaSource(); +// kafkaChannel.setBootstrapServers(endpoint); +// kafkaChannel.setTopic(topic); +// kafkaChannel.setGroupName(groupName); +// kafkaChannel.setJsonData(isJson); +// kafkaChannel.setMaxThread(maxThread); +// this.mainPipelineBuilder.setSource(kafkaChannel); +// return new DataStream(this.mainPipelineBuilder, null); +// } + public DataStream from(ISource source) { this.mainPipelineBuilder.setSource(source); return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null); diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java index c3c9f66a..8f9c315a 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java @@ -32,7 +32,7 @@ import org.apache.rocketmq.streams.client.transform.window.WindowInfo; import org.apache.rocketmq.streams.common.channel.impl.OutputPrintChannel; import org.apache.rocketmq.streams.common.channel.impl.file.FileSink; -import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.channel.source.AbstractSource; import org.apache.rocketmq.streams.common.channel.source.ISource; import org.apache.rocketmq.streams.common.component.ComponentCreator; @@ -54,6 +54,8 @@ import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; import org.apache.rocketmq.streams.common.topology.model.Union; import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; +import org.apache.rocketmq.streams.common.topology.stages.ShuffleConsumerChainStage; +import org.apache.rocketmq.streams.common.topology.stages.ShuffleProducerChainStage; import org.apache.rocketmq.streams.common.topology.stages.udf.StageBuilder; import org.apache.rocketmq.streams.common.topology.stages.udf.UDFChainStage; import org.apache.rocketmq.streams.common.topology.stages.udf.UDFUnionChainStage; @@ -97,6 +99,25 @@ public DataStream(PipelineBuilder pipelineBuilder, Set pipeline this.currentChainStage = currentChainStage; } + + public DataStream increaseConcurrencyByShuffle(int shuffleConcurrentCount){ + if(this.mainPipelineBuilder.getPipeline().getChannelNextStageLabel().size()>0||currentChainStage!=null){ + throw new RuntimeException("can only set after source"); + } + ChainStage stage = new ShuffleProducerChainStage(); + ((ShuffleProducerChainStage) stage).setShuffleOwnerName(MapKeyUtil.createKey(this.mainPipelineBuilder.getPipelineNameSpace(),this.mainPipelineBuilder.getPipelineName(),this.mainPipelineBuilder.getPipeline().getChannelName())); + ((ShuffleProducerChainStage) stage).setSplitCount(shuffleConcurrentCount); + this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); + + this.currentChainStage = stage; + + stage = new ShuffleConsumerChainStage<>(); + ((ShuffleConsumerChainStage) stage).setShuffleOwnerName(MapKeyUtil.createKey(this.mainPipelineBuilder.getPipelineNameSpace(),this.mainPipelineBuilder.getPipelineName(),this.mainPipelineBuilder.getPipeline().getChannelName())); + this.mainPipelineBuilder.setTopologyStages(currentChainStage, stage); + return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); + } + + public DataStream with(Strategy... strategies) { Properties properties = new Properties(); for (Strategy strategy : strategies) { @@ -177,11 +198,11 @@ protected T operate(IMessage message, AbstractContext context) { } else { List splitMessages = new ArrayList<>(); for (T t : result) { - Message subMessage = null; + Message subMessage = message.deepCopy(); if (t instanceof JSONObject) { - subMessage = new Message((JSONObject) t); + subMessage.setMessageBody((JSONObject) t); } else { - subMessage = new Message(new UserDefinedMessage(t)); + subMessage.setMessageBody(new UserDefinedMessage(t)); } splitMessages.add(subMessage); } @@ -361,7 +382,7 @@ public SplitStream split(SplitFunction splitFunction) { @Override protected T operate(IMessage message, AbstractContext context) { String labelName = splitFunction.split(message.getMessageValue()); - message.getHeader().addRouteLable(labelName); + message.getHeader().addRouteLabel(labelName); return null; } }; @@ -542,6 +563,13 @@ public DataStream toRocketmq(String topic, String tags, String groupName, int ba return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); } +// public DataStream toKafka(String bootstrapServers, String topic) { +// KafkaSink kafkaSink = new KafkaSink(bootstrapServers, topic); +// ChainStage output = this.mainPipelineBuilder.createStage(kafkaSink); +// this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); +// return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); +// } + public DataStream toEnhanceDBSink(String url, String userName, String password, String tableName) { EnhanceDBSink sink = new EnhanceDBSink(url, userName, password, tableName); ChainStage output = this.mainPipelineBuilder.createStage(sink); @@ -570,7 +598,7 @@ public DataStream toMqtt(String url, String clientId, String topic, String usern return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); } - public DataStream to(ISink sink) { + public DataStream to(AbstractSink sink) { ChainStage output = this.mainPipelineBuilder.createStage(sink); this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output); @@ -620,7 +648,6 @@ public void run() { } }); thread.start(); - } } else { pipeline.startChannel(); diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java index 4ef73b91..722a8c2a 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java @@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.rocketmq.streams.client.transform.window.Time; import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.model.NameCreatorContext; import org.apache.rocketmq.streams.common.topology.ChainStage; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; import org.apache.rocketmq.streams.common.utils.StringUtil; @@ -188,7 +189,7 @@ public Map createJoinFieldsFromCondition(String onCondition, Ato public static String createName(String functionName, String... names) { if (names == null || names.length == 0) { - return NameCreator.createNewName(INNER_VAR_NAME_PREFIX, functionName); + return NameCreatorContext.get().createNewName(INNER_VAR_NAME_PREFIX, functionName); } String[] values = new String[names.length + 2]; values[0] = INNER_VAR_NAME_PREFIX; @@ -196,6 +197,6 @@ public static String createName(String functionName, String... names) { for (int i = 2; i < values.length; i++) { values[i] = names[i - 2]; } - return NameCreator.createNewName(values); + return NameCreatorContext.get().createNewName(values); } } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/SplitStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/SplitStream.java index 7c6f4eb7..4d806ed2 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/SplitStream.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/SplitStream.java @@ -17,6 +17,7 @@ package org.apache.rocketmq.streams.client.transform; +import java.io.Serializable; import java.util.Set; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; @@ -24,8 +25,7 @@ import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; import org.apache.rocketmq.streams.common.topology.stages.udf.StageBuilder; -public class SplitStream { - +public class SplitStream implements Serializable { /** * 创建datastream时使用 @@ -34,7 +34,6 @@ public class SplitStream { protected Set otherPipelineBuilders; protected ChainStage currentChainStage; - public SplitStream(PipelineBuilder pipelineBuilder, Set pipelineBuilders, ChainStage currentChainStage) { this.pipelineBuilder = pipelineBuilder; this.otherPipelineBuilders = pipelineBuilders; @@ -43,18 +42,25 @@ public SplitStream(PipelineBuilder pipelineBuilder, Set pipelin /** * 选择一个分支 + * * @param lableName * @return */ - public DataStream select(String lableName){ - StageBuilder stage = new StageBuilder() { - @Override - protected T operate(IMessage message, AbstractContext context) { + public DataStream select(String lableName) { + StageBuilder operator = new StageBuilder() { + @Override protected T operate(IMessage message, AbstractContext context) { return null; } }; + + ChainStage stage = this.pipelineBuilder.createStage(operator); stage.setLabel(lableName); - this.pipelineBuilder.setTopologyStages(currentChainStage,stage); - return new DataStream(pipelineBuilder,otherPipelineBuilders,stage); + this.pipelineBuilder.setTopologyStages(currentChainStage, stage); + return new DataStream(pipelineBuilder, otherPipelineBuilders, stage); } + + public DataStream toDataStream() { + return new DataStream(pipelineBuilder, otherPipelineBuilders, currentChainStage); + } + } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java index 37abdf51..24fdf175 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java @@ -17,14 +17,36 @@ package org.apache.rocketmq.streams.client.transform; +import com.alibaba.fastjson.JSONObject; +import java.util.List; +import java.util.Map; +import java.util.Properties; import java.util.Set; +import javafx.util.Pair; import org.apache.rocketmq.streams.client.transform.window.Time; +import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; +import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.channel.source.AbstractSource; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.UserDefinedMessage; +import org.apache.rocketmq.streams.common.functions.MapFunction; import org.apache.rocketmq.streams.common.functions.ReduceFunction; +import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.model.NameCreatorContext; import org.apache.rocketmq.streams.common.topology.ChainStage; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; import org.apache.rocketmq.streams.common.topology.stages.udf.IReducer; +import org.apache.rocketmq.streams.common.utils.Base64Utils; +import org.apache.rocketmq.streams.common.utils.InstantiationUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.script.operator.impl.AggregationScript; +import org.apache.rocketmq.streams.script.service.IAccumulator; +import org.apache.rocketmq.streams.script.service.udf.SimpleUDAFScript; +import org.apache.rocketmq.streams.script.service.udf.UDAFScript; +import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent; +import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractWindow; /** @@ -90,10 +112,13 @@ public WindowStream distinct(String fieldName, String asName) { * @return */ public WindowStream count_distinct(String fieldName, String asName) { - String distinctName = "__" + fieldName + "_distinct_" + asName + "__"; - String prefix = distinctName + "=distinct(" + fieldName + ")"; - String suffix = asName + "=count(" + distinctName + ")"; - window.getSelectMap().put(asName, prefix + ";" + suffix); + return count_distinct_2(fieldName,asName); + } + + public WindowStream addUDAF(IAccumulator accumulator, String asName,String... fieldNames) { + AggregationScript.registUDAF(accumulator.getClass().getSimpleName(),accumulator.getClass()); + String prefix = asName + "="+accumulator.getClass().getSimpleName()+"(" + MapKeyUtil.createKeyBySign(",",fieldNames)+")"; + window.getSelectMap().put(asName,prefix); return this; } @@ -105,6 +130,23 @@ public WindowStream count_distinct_2(String fieldName, String asName) { return this; } + public WindowStream saveWindowMsg(MapFunction> mapFunction ,String sinkType, Properties properties) { + ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(IChannelBuilder.class.getName(), ServiceLoaderComponent.class); + IChannelBuilder builder = (IChannelBuilder) serviceLoaderComponent.loadService(sinkType.toLowerCase()); + + if (builder == null) { + throw new RuntimeException( + "expect channel creator for " + sinkType+ ". but not found"); + } + ISink sink= builder.createSink(pipelineBuilder.getPipelineNameSpace(), window.getConfigureName(), properties, null); + this.pipelineBuilder.addConfigurables(sink); + this.window.setContextMsgSinkName(sink.getConfigureName()); + byte[] bytes=InstantiationUtil.serializeObject(mapFunction); + String mapFunctionStr = Base64Utils.encode(bytes); + this.window.setMapFunctionSerializeValue(mapFunctionStr); + return this; + } + /** * count_distinct算子(数据量大,容忍较少错误率) * diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/WindowInfo.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/WindowInfo.java index e43e0943..73b543b3 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/WindowInfo.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/WindowInfo.java @@ -27,8 +27,7 @@ public class WindowInfo { public static int HOPPING_WINDOW = 1;//滑动窗口 public static int TUMBLING_WINDOW = 2;//滚动窗口 public static int SESSION_WINDOW = 3; - public static int OVER_WINDOW = 4; - public static int SHUFFLE_OVER_WINDOW = 5; + protected int type;//window类型 hopping,Tumbling protected Time windowSize;//窗口大小 protected Time windowSlide;//滑动大小 diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ApplicationTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ApplicationTest.java new file mode 100644 index 00000000..80595333 --- /dev/null +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ApplicationTest.java @@ -0,0 +1,57 @@ +/* + * 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.rocketmq.streams.client; + +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.topology.task.StreamsTask; +import org.apache.rocketmq.streams.configurable.ConfigurableComponent; +import org.junit.Test; + +public class ApplicationTest { + + @Test + public void testApplication() throws InterruptedException { + ComponentCreator.getProperties().put(ConfigureFileKey.POLLING_TIME, "5"); + ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE, "DB"); + ComponentCreator.getProperties().put("dipper.rds.jdbc.url", "jdbc:mysql://host:port/database?serverTimezone=Asia/Shanghai"); + ComponentCreator.getProperties().put("dipper.rds.jdbc.username", "username"); + ComponentCreator.getProperties().put("dipper.rds.jdbc.password", "password"); + + ConfigurableComponent configurableComponent = ComponentCreator.getComponent("chris_tmp", ConfigurableComponent.class); + StreamsTask streamsTask = configurableComponent.queryConfigurable(StreamsTask.TYPE, "task"); +// streamsTask.setNameSpace("chris_tmp"); +// streamsTask.setConfigureName("task"); + + if (streamsTask != null) { + StreamsTask copy = new StreamsTask(); + copy.toObject(streamsTask.toJson()); + + copy.setUpdateFlag(copy.getUpdateFlag() + 1); + configurableComponent.insert(copy); + StreamsTask streamsTask1 = configurableComponent.queryConfigurable(StreamsTask.TYPE, "task"); + System.out.println(streamsTask1.getUpdateFlag() == copy.getUpdateFlag()); + } + System.out.println(streamsTask.getUpdateFlag()); + // configurableComponent.refreshConfigurable("chris_tmp"); + // System.out.println(streamsTask.getUpdateFlag()); + while (true) { + Thread.sleep(1000); + + } + } +} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBDriverTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBDriverTest.java index 0894c873..8d6ada4c 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBDriverTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/DBDriverTest.java @@ -17,42 +17,25 @@ package org.apache.rocketmq.streams.client; -import org.apache.rocketmq.streams.common.component.ComponentCreator; -import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; import org.apache.rocketmq.streams.configurable.ConfigurableComponent; -import org.apache.rocketmq.streams.configurable.model.Configure; -import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.junit.Test; import static junit.framework.TestCase.assertNotNull; -/** - * 数据库的存储,需要配置存储的连接参数,请先完成配置,后执行单元用例 如果未建表,可以通过Configure.createTableSQL() 获取建表语句,创建表后,测试 - */ + public class DBDriverTest { - private String URL = ""; - protected String USER_NAME = ""; - protected String PASSWORD = ""; - protected String TABLE_NAME = "rocketmq_streams_configure_source"; @Test public void testDBConfigurableService() { String namespace = "streams.db.configurable"; - //正式使用时,在配置文件配置 - ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE, "DB"); - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL);//数据库连接url - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME);//用户名 - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD);//password - ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_TABLE_NAME, TABLE_NAME); - - //如果表不存在,创建表 - String sql = (Configure.createTableSQL(TABLE_NAME)); - DriverBuilder.createDriver().execute(sql); - ConfigurableComponent configurableComponent = ConfigurableComponent.getInstance(namespace); + ConfigurableComponent configurableComponent = ConfigurableComponent.getInstance("2211"); configurableComponent.insert(createPerson(namespace)); configurableComponent.refreshConfigurable(namespace); Person person = configurableComponent.queryConfigurable("person", "peronName"); + System.out.println(person.getName()); + System.out.println(person.getAge()); + assertNotNull(person); } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/MqttSourceExample.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/MqttSourceExample.java new file mode 100644 index 00000000..0c21055d --- /dev/null +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/MqttSourceExample.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.rocketmq.streams.client; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import com.google.common.collect.Sets; +import java.io.Serializable; +import java.util.Iterator; +import java.util.Set; +import org.apache.rocketmq.streams.client.source.DataStreamSource; +import org.apache.rocketmq.streams.client.strategy.ShuffleStrategy; +import org.apache.rocketmq.streams.client.transform.DataStream; +import org.apache.rocketmq.streams.client.transform.SplitStream; +import org.apache.rocketmq.streams.client.transform.window.Time; +import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.functions.SplitFunction; +import org.apache.rocketmq.streams.configurable.ConfigurableComponent; +import org.junit.Test; + +public class MqttSourceExample implements Serializable { + + public static void main(String[] args) throws InterruptedException { + DataStreamSource dataStream = DataStreamSource.create("namespace", "name"); + SplitStream ds = StreamBuilder.dataStream("tmp", "tmp") + .fromFile("window_msg_10.txt", true) + .split(new SplitFunction() { + @Override public String split(JSONObject o) { + return o.getString("ProjectName"); + } + }); + + ds.select("project-2").toPrint(); + ds.select("project-3").toPrint(); + ds.select("project-7").toPrint(); + ds.select("project-1").toPrint().start(); + } + + + @Test public void test1() { + DataStreamSource dataStreamSource = StreamBuilder.dataStream("", ""); + DataStream ds = dataStreamSource.fromMqtt("tcp://host:port", "", "", "", "") + .flatMap(message -> { + JSONObject obj = ((JSONObject) message); + JSONArray array = obj.getJSONArray("Data"); + String topic = obj.getString("__topic"); + for (int i = 0; i < array.size(); i++) { + array.getJSONObject(i).put("__topic", topic); + } + return array; + }); +// ds.toPrint().start(); + + SplitStream splitStream = ds.split(new SplitFunction() { + @Override public String split(JSONObject o) { + return o.getString("AttributeCode"); + } + }); + splitStream.select("Read").toPrint(); + splitStream.select("Write").toPrint().start(); + //splitStream.select("Write").toPrint().with(ShuffleStrategy.shuffleWithMemory()); + } + +} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ORMUtilTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ORMUtilTest.java index 9b649d5f..ab2961a0 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ORMUtilTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ORMUtilTest.java @@ -86,87 +86,3 @@ protected Person createPerson(String namespace,String name){ } -class Person extends BasedConfigurable { - @ENVDependence - private String name; - private int age; - private Boolean isMale; - private List addresses; - private Map childName2Age; - - public static Person createPerson(String namespace) { - Person person = new Person(); - person.setNameSpace(namespace); - person.setType("person"); - person.setConfigureName("Chris"); - person.setName("Chris"); - List addresses = new ArrayList<>(); - addresses.add("huilongguan"); - addresses.add("shangdi"); - person.setAddresses(addresses); - Map childName2Age = new HashMap<>(); - childName2Age.put("yuanyahan", 8); - childName2Age.put("yuanruxi", 4); - person.setChildName2Age(childName2Age); - person.setMale(true); - person.setAge(18); - return person; - } - - @Override - public String toString() { - return "org.apache.rocketmq.streams.Person{" + "name='" + name + '\'' + ", age=" + age + ", isMale=" + isMale + ", addresses=" + addresses - + ", childName2Age=" + childName2Age + '}'; - } - - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - public int getAge() { - return age; - } - - public void setAge(int age) { - this.age = age; - } - - public Boolean getMale() { - return isMale; - } - - public void setMale(Boolean male) { - isMale = male; - } - - public List getAddresses() { - return addresses; - } - - public void setAddresses(List addresses) { - this.addresses = addresses; - } - - public Map getChildName2Age() { - return childName2Age; - } - - public void setChildName2Age(Map childName2Age) { - this.childName2Age = childName2Age; - } - - @Override - public Object clone() { - Person person = null; - try { - person = (Person)super.clone(); - } catch (CloneNotSupportedException e) { - System.out.println("clone error " + e); - } - return person; - } -} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/Person.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/Person.java new file mode 100644 index 00000000..780d7d9d --- /dev/null +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/Person.java @@ -0,0 +1,89 @@ +package org.apache.rocketmq.streams.client; +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; + +import java.util.List; +import java.util.Map; + +public class Person extends BasedConfigurable { + @ENVDependence + private String name; + private int age; + private Boolean isMale; + private List addresses; + private Map childName2Age; + + @Override + public String toString() { + return "org.apache.rocketmq.streams.Person{" + "name='" + name + '\'' + ", age=" + age + ", isMale=" + isMale + ", addresses=" + addresses + + ", childName2Age=" + childName2Age + '}'; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public int getAge() { + return age; + } + + public void setAge(int age) { + this.age = age; + } + + public Boolean getMale() { + return isMale; + } + + public void setMale(Boolean male) { + isMale = male; + } + + public List getAddresses() { + return addresses; + } + + public void setAddresses(List addresses) { + this.addresses = addresses; + } + + public Map getChildName2Age() { + return childName2Age; + } + + public void setChildName2Age(Map childName2Age) { + this.childName2Age = childName2Age; + } + + @Override + public Object clone() { + Person person = null; + try { + person = (Person) super.clone(); + } catch (CloneNotSupportedException e) { + System.out.println("clone error " + e); + } + return person; + } +} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ScriptTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ScriptTest.java new file mode 100644 index 00000000..2ea9f2cd --- /dev/null +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/ScriptTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.client; + +import com.alibaba.fastjson.JSONObject; +import org.apache.rocketmq.streams.common.context.Context; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.script.ScriptComponent; +import org.junit.Test; + +public class ScriptTest { + @Test + public void testFloor(){ + JSONObject msg=new JSONObject(); + msg.put("_input",3233223.434334); + IMessage message=new Message(msg); + Context context=new Context(message); + ScriptComponent.getInstance().getService().executeScript(msg,"a=division(json_field(_input, 'lastTime'), 1000)"); + System.out.println(msg); + } +} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java index 89dd21b5..536e400e 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java @@ -43,6 +43,8 @@ public void testWindow() { .fromFile("/Users/duheng/project/opensource/sls_100.txt", false) .map((MapFunction) message -> JSONObject.parseObject(message)) .window(TumblingWindow.of(Time.seconds(5))) + .setTimeField("时间字段") + .waterMark(10) .groupBy("ProjectName", "LogStore") .setLocalStorageOnly(true) .count("total") diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/SplitTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/SplitTest.java new file mode 100644 index 00000000..5138be14 --- /dev/null +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/SplitTest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.client.example; + +import com.alibaba.fastjson.JSONObject; +import org.apache.rocketmq.streams.client.StreamBuilder; +import org.apache.rocketmq.streams.client.source.DataStreamSource; +import org.apache.rocketmq.streams.client.transform.SplitStream; +import org.apache.rocketmq.streams.common.functions.SplitFunction; + +public class SplitTest { + + public static void main(String[] args) throws InterruptedException { + DataStreamSource dataStream = DataStreamSource.create("namespace", "name"); + SplitStream ds = StreamBuilder.dataStream("tmp", "tmp") + .fromFile("window_msg_10.txt", true) + .split(new SplitFunction() { + @Override public String split(JSONObject o) { + return o.getString("ProjectName"); + } + }); + + ds.select("project-2").toPrint(); + ds.select("project-7").toPrint(); + ds.select("project-1").toPrint(); + + ds.toDataStream().start(); + + } + +} diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/sink/UserDefinedSink.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/sink/UserDefinedSink.java index 4a5ea8bb..d1e368be 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/sink/UserDefinedSink.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/sink/UserDefinedSink.java @@ -27,7 +27,7 @@ public class UserDefinedSink extends AbstractUDFSink { } - @Override protected void sendMessage2Store(ISplit split, List messageList) { + @Override protected void sendMessage2Store(ISplit split, List messageList) { } } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/sink/UserDefinedSupportShuffleSink.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/sink/UserDefinedSupportShuffleSink.java index 6c645111..9924c8dc 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/sink/UserDefinedSupportShuffleSink.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/sink/UserDefinedSupportShuffleSink.java @@ -30,7 +30,7 @@ public class UserDefinedSupportShuffleSink extends AbstractSupportShuffleUDFSink } - @Override public List getSplitList() { + @Override public List> getSplitList() { return null; } @@ -38,7 +38,7 @@ public class UserDefinedSupportShuffleSink extends AbstractSupportShuffleUDFSink } - @Override protected void sendMessage2Store(ISplit split, List messageList) { + @Override protected void sendMessage2Store(ISplit split, List messageList) { } } diff --git a/rocketmq-streams-commons/pom.xml b/rocketmq-streams-commons/pom.xml index 5ea155d4..2e73c6f8 100755 --- a/rocketmq-streams-commons/pom.xml +++ b/rocketmq-streams-commons/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 @@ -28,6 +30,11 @@ ROCKETMQ STREAMS :: commons jar + + org.slf4j + slf4j-log4j12 + ${slf4j-log4j12.version} + org.apache.commons @@ -52,17 +59,13 @@ fastjson - commons-logging - commons-logging + org.apache.httpcomponents + httpclient commons-io commons-io - - log4j - log4j - com.google.code.gson gson @@ -103,6 +106,35 @@ rocketmq-tools + + org.apache.rocketmq + rocketmq-client + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + + + + de.ruedigermoeller + fst + + + com.esotericsoftware + kryo + 5.3.0 + + + + commons-codec + commons-codec + 1.13 + diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/MappedByteBufferTableWithPrimaryIndex.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/MappedByteBufferTableWithPrimaryIndex.java new file mode 100644 index 00000000..72f2f892 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/MappedByteBufferTableWithPrimaryIndex.java @@ -0,0 +1,482 @@ +///* +// * Licensed to the Apache Software Foundation (ASF) under one or more +// * contributor license agreements. See the NOTICE file distributed with +// * this work for additional information regarding copyright ownership. +// * The ASF licenses this file to You under the Apache License, Version 2.0 +// * (the "License"); you may not use this file except in compliance with +// * the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ +//package org.apache.rocketmq.streams.common.cache; +// +//import java.io.File; +//import java.io.IOException; +//import java.io.RandomAccessFile; +//import java.io.Serializable; +//import java.nio.ByteOrder; +//import java.nio.MappedByteBuffer; +//import java.nio.channels.FileChannel; +//import java.util.*; +//import java.util.concurrent.atomic.AtomicInteger; +// +//import org.apache.commons.logging.Log; +//import org.apache.commons.logging.LogFactory; +//import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable; +//import org.apache.rocketmq.streams.common.utils.NumberUtils; +// +//@Deprecated +//public class MappedByteBufferTableWithPrimaryIndex extends AbstractMemoryTable { +// +// private static final Log logger = LogFactory.getLog(MappedByteBufferTableWithPrimaryIndex.class); +// +// protected String jobName; +// +// protected static final String mappedFilePrefix = "/tmp/dipper_"; +// +// protected static final int SIZE = 1024 * 1024 * 1024; +// +// protected int fileSize = -1; +// +// //1g 1个文件,超过1g,可以多个文件,每个文件对应一个MappedByteBuffer +// protected transient List caches = new ArrayList<>(); +// +// //当一个文件写满时,把文件的索引号和起始位置map起来,方便查找 +// //key = rowId, value为全局索引号, 把二维索引变成一维索引 +// protected transient Map maxOffsets = new HashMap<>(); +// +// //row id +// protected transient AtomicInteger currentIndex = new AtomicInteger(0); +// +// //全局游标 +// protected transient volatile long cursor = 0; +// +// protected transient volatile int currentFileIndex = 0; +// +// private static final int MAX_FILE_COUNT = 2; +// +// protected long totalMargin = 0; +// +// protected static final ByteOrder order = ByteOrder.nativeOrder(); +// +// public MappedByteBufferTableWithPrimaryIndex(){ +// +// } +// +// public MappedByteBufferTableWithPrimaryIndex(String jobName){ +// this.jobName = jobName; +// this.fileSize = SIZE; +// } +// +// public MappedByteBufferTableWithPrimaryIndex(String jobName, int fileSize){ +// this.jobName = jobName; +// this.fileSize = fileSize; +// } +// +// +// //todo file lock +// private final String createMappedFile(int fileIndex) throws IOException { +// String path = mappedFilePrefix + jobName + "_" + fileIndex; +// File file = new File(path); +// boolean isSuccess = file.exists(); +// if(isSuccess){ +// return path; +// }else{ +// isSuccess = file.createNewFile(); +// } +// if(isSuccess){ +// return path; +// }else{ +// logger.error(String.format("create mapped file error, file path is %s", path)); +// return null; +// } +// } +// +// private final MappedByteBuffer createMappedByteBuffer(int fileIndex) throws IOException { +// String filePath = createMappedFile(fileIndex); +// RandomAccessFile raf = new RandomAccessFile(filePath, "rw"); +// FileChannel fc = raf.getChannel(); +// MappedByteBuffer mbb = fc.map(FileChannel.MapMode.READ_WRITE, 0, getFileSizeOrDefault()); +// return mbb; +// } +// +// private final MappedByteBuffer getOrCreateMappedByteBuffer(int index) throws IOException { +// MappedByteBuffer buffer = null; +// if(index < caches.size()){ +// buffer = caches.get(index); +// } +// if(buffer == null){ +// buffer = createMappedByteBuffer(index); +// caches.add(buffer); +// } +// return buffer; +// } +// +// private final int calCurrentFileIndex(long cursor){ +// return Long.valueOf((cursor/getFileSizeOrDefault())).intValue(); +// } +// +// private final int calCurrentFileIndex(){ +// return calCurrentFileIndex(cursor); +// } +// +// private final FilePosition seek(long startIndex){ +// +// int fileIndex = Long.valueOf(startIndex / getFileSizeOrDefault()).intValue(); +// long bufferPosition = startIndex % getFileSizeOrDefault(); +// return new FilePosition(fileIndex, bufferPosition); +// +// } +// +//// 1. 在 tmp目录创建文件,一个文件1g,最大4g(2g?) +//// 2. 一台物理机/虚拟机可能运行多个taskmanager,一个taskmanager可能会有多个并发,文件目录 为 tmp/${jobName}/00 ~ 03 +//// 3. 初始化加写锁? +//// 4. index是什么意思? +// +// @Override +// public Iterator newIterator() { +// return new Iterator() { +// +// protected int rowIndex = 0; +// private final int count = maxOffsets.size(); +// +// @Override +// public boolean hasNext() { +// check(maxOffsets.size()); +// return rowIndex < count; +// } +// +// @Override +// public RowElement next() { +// check(maxOffsets.size()); +// Map row = getRow(rowIndex); +// return new RowElement(row, rowIndex++); +// } +// +// private final void check(int count){ +// if(this.count != count){ +// throw new ConcurrentModificationException("unsupported modified. " + this.count + " != " + count); +// } +// } +// }; +// } +// +// /** +// * 保存row到文件,整个数据的构成:行长度(2个字节)+ 行字节数组 +// * 每行多列:每一列前面加一个长度,标识这个列代表的字节数组的长度 +// * 长度用1-2个字节表示,根据第一位是0还是1,决定是1个字节还是两个字节 +// * +// * @param values +// * @return row id +// */ +// @Override +// protected Integer saveRowByte(byte[][] values, int byteSize) { +// int size = 0; +// List byteList = new ArrayList(); +// for(byte[] bytes : values){ +// byte[] lenBytes = createLenByte(bytes); +// byte[] columnBytes = new byte[lenBytes.length + bytes.length]; +// int index = 0; +// for(byte b : lenBytes){ +// columnBytes[index] = b; +// index++; +// } +// for(byte b : bytes){ +// columnBytes[index] = b; +// index++; +// } +// byteList.add(columnBytes); +// size += columnBytes.length; +// } +//// size = size + 2; +// //计算整行的长度 +// byte[] rowLength = createLenByte(size); +// int startIndex = currentIndex.getAndIncrement(); +// //指向下一个 +// byteList.add(0, rowLength);//加上长度 +// size = size + rowLength.length; +// try { +// //写入成功之后再加size +// save2File(startIndex, byteList, size); +// } catch (IOException e) { +// e.printStackTrace(); +// } +// return startIndex; +// } +// +// /** +// * 从文件中加载行 +// * +// * @param startIndex +// * @return +// */ +// @Override +// protected byte[][] loadRowByte(Integer startIndex) { +// +// List bytes = null; +// try { +// bytes = getFromFile(startIndex); +// } catch (IOException e) { +// e.printStackTrace(); +// } +// byte[][] byteRows = new byte[bytes.size()][]; +// int i = 0; +// for(byte[] byteArray : bytes){ +// byteRows[i] = byteArray; +// i++; +// } +// return byteRows; +// +// } +// +// /** +// * startIndex 为row id +// * @param startIndex +// * @return +// * @throws IOException +// */ +// protected List getFromFile(int startIndex) throws IOException { +// +// long cursor = maxOffsets.get(startIndex); +// FilePosition position = seek(cursor); +// int fileIndex = position.fileIndex; +// long bufferCursor = position.bufferPosition; +// MappedByteBuffer buffer = caches.get(fileIndex); +// //解析整行的长度 +// byte[] rowLenBytes = readLen((int)bufferCursor, buffer); +// //计算整行的长度, 不包含行长度的元数据 +// int length = readLenFromBytes(rowLenBytes); +// //解析出整行的数据 +// byte[] bytes = readByteFromFile(buffer, bufferCursor + rowLenBytes.length, length); +// Listresult = new ArrayList<>(); +// int index = 0; +// while (index < length){ +// byte[] lenBytes = readLen(bytes, index); +// index = index + lenBytes.length; +// int len = readLenFromBytes(lenBytes); +// byte[] column = readByteFromBytes(bytes, index, len); +// result.add(column); +// index = index + column.length; +// } +// return result; +// } +// +// +// private byte[] readByteFromBytes(byte[] bytes,int index, int len){ +//// if(len == 0){ +//// return new byte[0]; +//// } +// byte[] ret = new byte[len]; +// for(int i = 0; i < len; i++){ +// ret[i] = bytes[index++]; +// } +// return ret; +// } +// +// /** +// * 从字节数组解析长度 +// * @param bytes +// * @return +// */ +// private int readLenFromBytes(byte[] bytes){ +// int bytesLength = bytes.length; +// assert (bytesLength == 1 || bytesLength == 2) : "bytes length must be 1 or 2, but " + Arrays.toString(bytes); +// if(bytesLength == 1){ +// return NumberUtils.toInt(bytes[0]); +// }else{ +// bytes[1] = (byte)(bytes[1] & 0x7f); +// return NumberUtils.toInt(bytes); +// } +// } +// +// private byte[] readLen(int start, MappedByteBuffer buffer){ +// buffer.position(0); +// byte b = buffer.get(start); +// int length = 1; +// if((b & 0x80) == 0x80){ +// length = 2; +// } +// byte[] ret = new byte[length]; +// ret[0] = b; +// if(length == 2){ +// ret[1] = buffer.get(); +// } +//// buffer.position(0); +//// System.out.println("start - " + start + ", " + "remaing - " + buffer.remaining() + ", " + "position - " + buffer.position()); +//// buffer.get(ret, start, length); +// return ret; +// } +// +// private byte[] readLen(byte[] bytes, int index) { +// byte b = bytes[index]; +// //第一位为1 +// if((b & 0x80) == 0x80){ +// byte[] ret = new byte[2]; +// ret[0] = b; +// ret[1] = bytes[++index]; +// return ret; +// }else{ +// byte[] ret = new byte[1]; +// ret[0] = b; +// return ret; +// } +// } +// +// private byte[] readByteFromFile(MappedByteBuffer buffer, long startIndex, int len) throws IOException { +// byte[] dst = new byte[len]; +// int index = Long.valueOf(startIndex).intValue(); +// buffer.position(index); +// buffer.get(dst); +// return dst; +// } +// +// /** +// * startIndex 为row id +// * @param startIndex +// * @param list +// * @param byteSize +// * @throws IOException +// */ +// private void save2File(int startIndex, List list, int byteSize) throws IOException { +// MappedByteBuffer mappedByteBuffer = getOrCreateMappedByteBufferWithCapacity(byteSize); +// if(byteSize > getFileSizeOrDefault()){ +// throw new RuntimeException("row is too long, size is " + byteSize); +// } +// for(byte[] bytes : list){ +// mappedByteBuffer.put(bytes); +// } +// maxOffsets.put(startIndex, cursor); +//// if((maxOffsets.size() % 100000) == 1){ +//// System.out.println(maxOffsets.size()); +//// } +// cursor = cursor + byteSize; +// } +// +// /** +// * 获取数组的长度, 大于等于128的用两个byte表示, 小于等于127的用一个byte表示 +// * @param bytes +// * @return +// */ +// private byte[] createLenByte(byte[] bytes) { +// //todo 如果长度小于127,用一个字节表示长度,第一位为0 +// //如果长度大于127,用两个字节表示长度,第一位为1 +// //返回标识长度的字节数组 +// return createLenByte(bytes.length); +// } +// +// private byte[] createLenByte(int size) { +// byte[] ret; +// int length = size; +// //max length 65535/2 +// if(length >= 0 && length <= 127){ +// ret = NumberUtils.toByteArray(length, 0xff); +// }else if(length >= 128 && length <= 32767){ +// ret = NumberUtils.toByteArray(length, 0xffff); +// //第一位置成1 +// ret[1] = (byte)((ret[1]) ^ ((byte)0x80)); +// }else{ +// throw new RuntimeException("length must be 1 ~ 32767, but " + size); +// } +// return ret; +// } +// +// /** +// * +// * @param fileIndex +// * @return +// * @throws IOException +// */ +// private MappedByteBuffer selectFile(int fileIndex) throws IOException { +//// Long cursor = maxOffsets.get(startIndex); +//// int fileIndex = calCurrentFileIndex(cursor.longValue()); +// MappedByteBuffer mappedByteBuffer = caches.get(fileIndex); +// return mappedByteBuffer; +// } +// +// /** +// * 检查当前的mappedByteBuffer 是否有足够的容量 +// * @param byteSize +// * @return +// * @throws IOException +// */ +// private MappedByteBuffer getOrCreateMappedByteBufferWithCapacity(int byteSize) throws IOException { +// +// int tmpFileIndex = calCurrentFileIndex(); +// MappedByteBuffer buffer = getOrCreateMappedByteBuffer(tmpFileIndex); +// int leftCapacity = getFileSizeOrDefault() - (buffer.position() + 1); +// if(leftCapacity <= byteSize){ +// tmpFileIndex += 1; +// totalMargin = totalMargin + leftCapacity; +// cursor = cursor + leftCapacity; +// buffer = getOrCreateMappedByteBuffer(tmpFileIndex); +// } +// return buffer; +// } +// +// public String getJobName() { +// return jobName; +// } +// +// public void setJobName(String jobName) { +// this.jobName = jobName; +// } +// +// public int getFileSize() { +// return fileSize; +// } +// +// public void setFileSize(int fileSize) { +// this.fileSize = fileSize; +// } +// +// private final int getFileSizeOrDefault(){ +// if(fileSize == -1){ +// fileSize = SIZE; +// } +// return fileSize; +// } +// +// public List getCaches() { +// return caches; +// } +// +// public void setCaches(List caches) { +// this.caches = caches; +// } +// +// public static class FilePosition implements Serializable{ +// +// private static final long serialVersionUID = 9128119471875329716L; +// +// public int fileIndex; +// public long bufferPosition; +// +// public FilePosition(int a, long b){ +// fileIndex = a; +// bufferPosition = b; +// } +// +// public int getFileIndex() { +// return fileIndex; +// } +// +// public void setFileIndex(int fileIndex) { +// this.fileIndex = fileIndex; +// } +// +// public long getBufferPosition() { +// return bufferPosition; +// } +// +// public void setBufferPosition(long bufferPosition) { +// this.bufferPosition = bufferPosition; +// } +// +// } +//} \ No newline at end of file diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVAddress.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVAddress.java index bd7d2451..912a7dd2 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVAddress.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVAddress.java @@ -74,19 +74,19 @@ public KVAddress(ByteArray byteArray) { return; } - byte[] bytes = new byte[2]; - bytes[0] = byteArray.getByte(3); - int x = conflictValue & 127; - bytes[1] = NumberUtils.toByte(x)[0]; + byte[] bytes=new byte[2]; + bytes[0]=byteArray.getByte(3); + int x=conflictValue&127; + bytes[1]= NumberUtils.toByte(x)[0]; this.conflictIndex = NumberUtils.toInt(bytes); this.offset = byteArray.castInt(0, 3); } public byte[] createBytes() { - byte[] values = NumberUtils.toByte(offset, 5); - byte[] indexBytes = NumberUtils.toByte(conflictIndex); - values[3] = indexBytes[0]; - byte fisrtByte = indexBytes[1]; + byte[] values = NumberUtils.toByte(offset,5); + byte[] indexBytes= NumberUtils.toByte(conflictIndex); + values[3]=indexBytes[0]; + byte fisrtByte =indexBytes[1]; int value = 0; if (isConflict) { @@ -96,36 +96,36 @@ public byte[] createBytes() { } values[4] = (byte) (value & 0xff); - return values; + return values; } public byte[] createBytesIngoreFirstBit() { - byte[] values = NumberUtils.toByte(offset, 5); - byte[] indexBytes = NumberUtils.toByte(conflictIndex); - values[3] = indexBytes[0]; - values[4] = indexBytes[1]; + byte[] values = NumberUtils.toByte(offset,5); + byte[] indexBytes= NumberUtils.toByte(conflictIndex); + values[3]=indexBytes[0]; + values[4]=indexBytes[1]; return values; } - public Long convertLongValue() { - byte[] bytes = createBytesIngoreFirstBit(); + public Long convertLongValue(){ + byte[] bytes=createBytesIngoreFirstBit(); return NumberUtils.toLong(bytes); } - public static KVAddress createMapAddressFromLongValue(Long value) { - byte[] bytes = NumberUtils.toByte(value); + public static KVAddress createMapAddressFromLongValue(Long value){ + byte[] bytes=NumberUtils.toByte(value); return createAddressFromBytes(bytes); } - public static KVAddress createAddressFromBytes(byte[] bytes) { - int offset = NumberUtils.toInt(bytes, 0, 3); - byte firstByte = bytes[4]; - if (firstByte < 0) { - bytes[4] = 0; + public static KVAddress createAddressFromBytes(byte[] bytes){ + int offset=NumberUtils.toInt(bytes,0,3); + byte firstByte =bytes[4]; + if(firstByte<0){ + bytes[4]=0; } - int index = NumberUtils.toInt(bytes, 3, 2); - KVAddress mapAddress = new KVAddress(index, offset); + int index=NumberUtils.toInt(bytes,3,2); + KVAddress mapAddress=new KVAddress(index,offset); return mapAddress; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/AbstractChannel.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/AbstractChannel.java index 313ac7f1..ca04d714 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/AbstractChannel.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/AbstractChannel.java @@ -27,6 +27,7 @@ import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.MessageOffset; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; @@ -73,11 +74,16 @@ public void getJsonObject(JSONObject jsonObject) { create(); if (sourceValue != null) { source = InstantiationUtil.deserializeObject(Base64Utils.decode(sourceValue)); - source.init(); + if(source!=null){ + source.init(); + } + } if (sinkValue != null) { sink = InstantiationUtil.deserializeObject(Base64Utils.decode(sinkValue)); - sink.init(); + if(sink!=null){ + sink.init(); + } } } @@ -150,7 +156,7 @@ public ISource getSource() { } @Override - public boolean batchAdd(IMessage fieldName2Value, ISplit split) { + public boolean batchAdd(IMessage fieldName2Value, ISplit split) { return sink.batchAdd(fieldName2Value, split); } @@ -254,4 +260,6 @@ public void atomicSink(ISystemMessage message){ } + + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractSupportShuffleChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractSupportShuffleChannelBuilder.java index c2f1d455..cc149b8c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractSupportShuffleChannelBuilder.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/AbstractSupportShuffleChannelBuilder.java @@ -24,7 +24,7 @@ public abstract class AbstractSupportShuffleChannelBuilder implements IChannelBuilder, IShuffleChannelBuilder { @Override - public ISource copy(ISource pipelineSource) { + public ISource copy(ISource pipelineSource) { JSONObject jsonObject = JSONObject.parseObject(pipelineSource.toJson()); return ConfigurableUtil.create(pipelineSource.getNameSpace(), pipelineSource.getConfigureName(), jsonObject, pipelineSource.getClass().getName()); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IChannelBuilder.java index 27ba3fda..f4a7dd96 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IChannelBuilder.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IChannelBuilder.java @@ -31,13 +31,12 @@ public interface IChannelBuilder { /** * 主要完成sql中的with 属性和source/sink字段名的映射 * - * @param formatProperties - * @param inputProperties - * @param formatName - * @param inputName + * @param formatProperties 资源文件 + * @param inputProperties 资源文件 + * @param formatName key值 + * @param inputName 输入值 */ - static void formatPropertiesName(JSONObject formatProperties, Properties inputProperties, String formatName, - String inputName) { + static void formatPropertiesName(JSONObject formatProperties, Properties inputProperties, String formatName, String inputName) { String inputValue = inputProperties.getProperty(inputName); if (StringUtil.isNotEmpty(inputValue)) { formatProperties.put(formatName, inputValue); @@ -47,24 +46,24 @@ static void formatPropertiesName(JSONObject formatProperties, Properties inputPr /** * 创建channel * - * @param properties - * @return + * @param properties 资源文件 + * @return source实例 */ - ISource createSource(String namespace, String name, Properties properties, MetaData metaData); + ISource createSource(String namespace, String name, Properties properties, MetaData metaData); /** * 返回channel类型,和blink语句中的type值一致 * - * @return + * @return 类型 */ String getType(); /** * 创建channel * - * @param properties - * @return + * @param properties 资源文件 + * @return sink实例 */ - ISink createSink(String namespace, String name, Properties properties, MetaData metaData); + ISink createSink(String namespace, String name, Properties properties, MetaData metaData); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IShuffleChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IShuffleChannelBuilder.java index a6e64f4c..a03684f1 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IShuffleChannelBuilder.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/builder/IShuffleChannelBuilder.java @@ -27,7 +27,7 @@ public interface IShuffleChannelBuilder { * @param pipelineSource * @return */ - ISource copy(ISource pipelineSource); + ISource copy(ISource pipelineSource); /** * 根据数据源source 创建一个source shuffle出来 @@ -35,5 +35,5 @@ public interface IShuffleChannelBuilder { * @param pipelineSource * @return */ - ISink createBySource(ISource pipelineSource); + ISink createBySource(ISource pipelineSource); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/CollectionSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/CollectionSink.java new file mode 100644 index 00000000..d3431ccc --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/CollectionSink.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.common.channel.impl; + +import java.util.ArrayList; +import java.util.List; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; +import org.apache.rocketmq.streams.common.context.IMessage; + +/** + * @description just support json object + */ +public class CollectionSink extends AbstractSink { + + List data; + + public CollectionSink() { + data = new ArrayList<>(); + } + + public CollectionSink(List data) { + this.data = data; + } + + @Override + protected synchronized boolean batchInsert(List messages) { + for (IMessage msg : messages) { + if (msg.isJsonMessage()) { + data.add(msg.getMessageBody().toJSONString()); + } else { + data.add(msg.getMessageValue()); + } + } + + return false; + } + + public List getData() { + return data; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/CollectionSinkBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/CollectionSinkBuilder.java new file mode 100644 index 00000000..f2d67a0f --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/CollectionSinkBuilder.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.common.channel.impl; + +import com.google.auto.service.AutoService; +import java.util.Properties; +import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; +import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.model.ServiceName; + +/** + * @description + */ +@AutoService(IChannelBuilder.class) +@ServiceName(value = CollectionSinkBuilder.TYPE) +public class CollectionSinkBuilder implements IChannelBuilder { + + public static final String TYPE = "collection"; + + @Override + public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + return new CollectionSource(); + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + return new CollectionSink(); + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/PrintChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/PrintChannelBuilder.java new file mode 100644 index 00000000..a48e9307 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/PrintChannelBuilder.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.common.channel.impl; + +import com.google.auto.service.AutoService; +import java.util.Properties; +import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; +import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.model.ServiceName; + +@AutoService(IChannelBuilder.class) +@ServiceName(value = PrintChannelBuilder.TYPE, aliasName = "OutputPrintChannel") +public class PrintChannelBuilder implements IChannelBuilder { + public static final String TYPE = "print"; + + @Override + public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + throw new RuntimeException("can not support this method"); + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + return new OutputPrintChannel(); + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileChannelBuilder.java new file mode 100644 index 00000000..eb846890 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileChannelBuilder.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.rocketmq.streams.common.channel.impl.file; + +import com.alibaba.fastjson.JSONObject; +import com.google.auto.service.AutoService; +import java.util.Properties; +import org.apache.rocketmq.streams.common.channel.builder.AbstractSupportShuffleChannelBuilder; +import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; +import org.apache.rocketmq.streams.common.channel.impl.memory.MemoryChannel; +import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.model.ServiceName; +import org.apache.rocketmq.streams.common.utils.ConfigurableUtil; + +@AutoService(IChannelBuilder.class) +@ServiceName(value = FileChannelBuilder.TYPE, aliasName = "FileSource") +public class FileChannelBuilder extends AbstractSupportShuffleChannelBuilder { + + public static final String TYPE = "file"; + + /** + * @param namespace + * @param name + * @param properties + * @return + */ + @Override + public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + + FileSource fileSource = (FileSource) ConfigurableUtil.create(FileSource.class.getName(), namespace, name, createFormatProperty(properties), null); + return fileSource; + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + FileSink fileSink = (FileSink) ConfigurableUtil.create(FileSink.class.getName(), namespace, name, createFormatProperty(properties), null); + return fileSink; + } + + /** + * 创建标准的属性文件 + * + * @param properties + * @return + */ + @Override + protected JSONObject createFormatProperty(Properties properties) { + JSONObject formatProperties = new JSONObject(); + for (Object object : properties.keySet()) { + String key = (String) object; + if ("type".equals(key)) { + continue; + } + formatProperties.put(key, properties.get(key)); + } + IChannelBuilder.formatPropertiesName(formatProperties, properties, "filePath", "filePath"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "filePath", "filepath"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "filePath", "fileName"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "thread.max.count"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "maxthread"); + return formatProperties; + } + + @Override + public ISource copy(ISource pipelineSource) { + return new MemoryChannel(); + } + + @Override + public ISink createBySource(ISource pipelineSource) { + return new MemoryChannel(); + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSink.java index 1a8ac009..93f727a4 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSink.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSink.java @@ -74,9 +74,9 @@ protected void createTopicIfNotExist(int splitNum) { } @Override - public List getSplitList() { + public List> getSplitList() { File file = new File(filePath); - List splits = new ArrayList<>(); + List> splits = new ArrayList<>(); splits.add(new FileSplit(file)); return splits; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSource.java index 61aadd79..2843abff 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSource.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/file/FileSource.java @@ -116,7 +116,7 @@ protected LinkedBlockingQueue createIteratorList() { LinkedBlockingQueue iterators = new LinkedBlockingQueue<>(1000); File file = getFile(filePath); if (!file.exists()) { - throw new RuntimeException("filePath not exist.the filePath is "+filePath); + return null; } try { if (!file.isDirectory()) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryChannelBuilder.java new file mode 100644 index 00000000..720e3e17 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemoryChannelBuilder.java @@ -0,0 +1,64 @@ +/* + * 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.rocketmq.streams.common.channel.impl.memory; + +import com.google.auto.service.AutoService; +import java.util.Properties; +import org.apache.rocketmq.streams.common.channel.builder.AbstractSupportShuffleChannelBuilder; +import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; +import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.model.ServiceName; + +@AutoService(IChannelBuilder.class) +@ServiceName(value = MemoryChannelBuilder.TYPE, aliasName = "MemorySource") +public class MemoryChannelBuilder extends AbstractSupportShuffleChannelBuilder { + + public static final String TYPE = "memory"; + + /** + * @param namespace + * @param name + * @param properties + * @return + */ + @Override + public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + return new MemorySource(); + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + return new MemorySink(); + } + + @Override + public ISource copy(ISource pipelineSource) { + return new MemoryChannel(); + } + + @Override + public ISink createBySource(ISource pipelineSource) { + return new MemoryChannel(); + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySink.java index 213364f6..618dbb83 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySink.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/memory/MemorySink.java @@ -74,8 +74,8 @@ protected void createTopicIfNotExist(int splitNum) { } @Override - public List getSplitList() { - List splits = new ArrayList<>(); + public List> getSplitList() { + List> splits = new ArrayList<>(); splits.add(new MemorySplit()); return splits; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewChannelBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewChannelBuilder.java new file mode 100644 index 00000000..cb644cb6 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewChannelBuilder.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.streams.common.channel.impl.view; + +import com.alibaba.fastjson.JSONObject; +import com.google.auto.service.AutoService; +import java.util.Properties; +import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; +import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.model.ServiceName; +import org.apache.rocketmq.streams.common.utils.ConfigurableUtil; + +@AutoService(IChannelBuilder.class) +@ServiceName(value = ViewChannelBuilder.TYPE, aliasName = "view") +public class ViewChannelBuilder implements IChannelBuilder { + public static final String TYPE = "view"; + + @Override public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) { + return (ViewSource) ConfigurableUtil.create(ViewSource.class.getName(), namespace, name, createFormatProperty(properties), null); + } + + @Override public String getType() { + return TYPE; + } + + @Override public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { + return (ViewSink) ConfigurableUtil.create(ViewSink.class.getName(), namespace, name, createFormatProperty(properties), null); + } + + /** + * 创建标准的属性文件 + * + * @param properties + * @return + */ + protected JSONObject createFormatProperty(Properties properties) { + JSONObject formatProperties = new JSONObject(); + for (Object object : properties.keySet()) { + String key = (String) object; + if ("type".equals(key)) { + continue; + } + formatProperties.put(key, properties.getProperty(key)); + } + IChannelBuilder.formatPropertiesName(formatProperties, properties, "viewTableName", "tableName"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "viewTableName", "viewName"); + IChannelBuilder.formatPropertiesName(formatProperties, properties, "viewTableName", "name"); +// IChannelBuilder.formatPropertiesName(formatProperties, properties, "maxThread", "thread.max.count"); + return formatProperties; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSink.java new file mode 100644 index 00000000..82265d47 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSink.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.rocketmq.streams.common.channel.impl.view; + +import java.util.List; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; +import org.apache.rocketmq.streams.common.context.IMessage; + +public class ViewSink extends AbstractSink { + private static final Log LOG = LogFactory.getLog(ViewSink.class); + + protected String viewTableName; + @Override protected boolean batchInsert(List messages) { + return false; + } + + public String getViewTableName() { + return viewTableName; + } + + public void setViewTableName(String viewTableName) { + this.viewTableName = viewTableName; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSource.java new file mode 100644 index 00000000..710fefed --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/view/ViewSource.java @@ -0,0 +1,66 @@ +/* + * 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.rocketmq.streams.common.channel.impl.view; + +import org.apache.rocketmq.streams.common.channel.source.AbstractSource; +import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; +import org.apache.rocketmq.streams.common.topology.task.TaskAssigner; + +public class ViewSource extends AbstractSource { + protected String tableName; + + + @Override + public void addConfigurables(PipelineBuilder pipelineBuilder) { + TaskAssigner taskAssigner=new TaskAssigner(); + taskAssigner.setTaskName(tableName); + taskAssigner.setPipelineName(pipelineBuilder.getPipelineName());; + pipelineBuilder.addConfigurables(taskAssigner); + pipelineBuilder.addConfigurables(this); + } + + + + @Override protected boolean startSource() { + return true; + } + + @Override public boolean supportNewSplitFind() { + return false; + } + + @Override public boolean supportRemoveSplitFind() { + return false; + } + + @Override public boolean supportOffsetRest() { + return false; + } + + @Override protected boolean isNotDataSplit(String queueId) { + return false; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSink.java index f0437289..3b4c8317 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSink.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSink.java @@ -35,6 +35,7 @@ import org.apache.rocketmq.streams.common.checkpoint.SourceState; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; import org.apache.rocketmq.streams.common.configurable.IConfigurableIdentification; +import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.MessageOffset; import org.apache.rocketmq.streams.common.interfaces.ILifeCycle; @@ -71,18 +72,18 @@ protected boolean initConfigurable() { } @Override - public boolean batchAdd(IMessage fieldName2Value, ISplit split) { - fieldName2Value.getMessageBody().put(TARGET_QUEUE, split); - return batchAdd(fieldName2Value); + public boolean batchAdd(IMessage message, ISplit split) { + message.getMessageBody().put(TARGET_QUEUE, split); + return batchAdd(message); } - public ISplit getSplit(IMessage message) { - return (ISplit) message.getMessageBody().get(TARGET_QUEUE); + public ISplit getSplit(IMessage message) { + return (ISplit) message.getMessageBody().get(TARGET_QUEUE); } @Override - public boolean batchAdd(IMessage fieldName2Value) { - messageCache.addCache(fieldName2Value); + public boolean batchAdd(IMessage message) { + messageCache.addCache(message); return true; } @@ -164,7 +165,7 @@ public boolean flushMessage(List messages) { if (source == null) { continue; } - String pipelineName = message.getHeader().getPiplineName(); + String pipelineName = message.getHeader().getPipelineName(); String sourceName = CheckPointManager.createSourceName(source, pipelineName); SourceState sourceState = this.sourceName2State.get(sourceName); if (sourceState == null) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSupportShuffleSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSupportShuffleSink.java index 5a19bfbd..bb59f505 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSupportShuffleSink.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSupportShuffleSink.java @@ -21,7 +21,7 @@ public abstract class AbstractSupportShuffleSink extends AbstractSink { - protected transient int splitNum ;//分片个数 + protected transient int splitNum;//分片个数 //sls对应的project和logstore初始化是否完成标志 protected volatile transient boolean hasCreated = false; @@ -43,13 +43,13 @@ public abstract class AbstractSupportShuffleSink extends AbstractSink { * * @return */ - public abstract List getSplitList(); + public abstract List> getSplitList(); @Override protected boolean initConfigurable() { boolean success = super.initConfigurable(); hasCreated = false; - if(this.splitNum>0){ + if (this.splitNum > 0) { checkAndCreateTopic(); } return success; @@ -64,7 +64,7 @@ protected void checkAndCreateTopic() { if (!hasCreated) { try { createTopicIfNotExist(splitNum); - }catch (Exception e){ + } catch (Exception e) { e.printStackTrace(); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSupportShuffleUDFSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSupportShuffleUDFSink.java index e754a864..d5ef4f8c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSupportShuffleUDFSink.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractSupportShuffleUDFSink.java @@ -29,7 +29,7 @@ public abstract class AbstractSupportShuffleUDFSink extends AbstractSupportShuff sink.sendMessage2Store(messageList); } - @Override protected void sendMessage2Store(ISplit split, List messageList) { + @Override protected void sendMessage2Store(ISplit split, List messageList) { sink.sendMessage2Store(split, messageList); } }); @@ -37,7 +37,7 @@ public abstract class AbstractSupportShuffleUDFSink extends AbstractSupportShuff @Override public int getSplitNum() { - List splits = getSplitList(); + List> splits = getSplitList(); if (splits == null || splits.size() == 0) { return 0; } @@ -46,5 +46,5 @@ public int getSplitNum() { protected abstract void sendMessage2Store(List messageList); - protected abstract void sendMessage2Store(ISplit split, List messageList); + protected abstract void sendMessage2Store(ISplit split, List messageList); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractUDFSink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractUDFSink.java index a390cfbf..8aee4622 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractUDFSink.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/AbstractUDFSink.java @@ -23,31 +23,28 @@ import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.context.IMessage; -public abstract class AbstractUDFSink extends AbstractSink{ +public abstract class AbstractUDFSink extends AbstractSink { @Override protected boolean batchInsert(List messages) { - return batchInsert(messages,this); + return batchInsert(messages, this); } - public static boolean batchInsert(List messages,AbstractUDFSink sink) { + + public static boolean batchInsert(List messages, AbstractUDFSink sink) { if (messages == null) { return true; } try { Map> msgsByQueueId = new HashMap<>();// group by queueId, if the message not contains queue info ,the set default string as default queueId - Map messageQueueMap = new HashMap<>();//if has queue id in message, save the map for queueid 2 messagequeeue + Map> messageQueueMap = new HashMap<>();//if has queue id in message, save the map for queueid 2 messagequeeue String defaultQueueId = "";//message is not contains queue ,use default for (IMessage msg : messages) { - ISplit channelQueue = sink.getSplit(msg); + ISplit channelQueue = sink.getSplit(msg); String queueId = defaultQueueId; if (channelQueue != null) { queueId = channelQueue.getQueueId(); messageQueueMap.put(queueId, channelQueue); } - List messageList = msgsByQueueId.get(queueId); - if (messageList == null) { - messageList = new ArrayList<>(); - msgsByQueueId.put(queueId, messageList); - } + List messageList = msgsByQueueId.computeIfAbsent(queueId, k -> new ArrayList<>()); messageList.add(msg); } List messageList = msgsByQueueId.get(defaultQueueId); @@ -60,8 +57,8 @@ public static boolean batchInsert(List messages,AbstractUDFSink sink) } for (String queueId : msgsByQueueId.keySet()) { messageList = msgsByQueueId.get(queueId); - ISplit split=messageQueueMap.get(queueId); - sink.sendMessage2Store(split,messageList); + ISplit split = messageQueueMap.get(queueId); + sink.sendMessage2Store(split, messageList); } } catch (Exception e) { @@ -74,5 +71,5 @@ public static boolean batchInsert(List messages,AbstractUDFSink sink) protected abstract void sendMessage2Store(List messageList); - protected abstract void sendMessage2Store(ISplit split,List messageList); + protected abstract void sendMessage2Store(ISplit split, List messageList); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/ISink.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/ISink.java index e95b4704..fb3d055d 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/ISink.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sink/ISink.java @@ -23,6 +23,7 @@ import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage; import org.apache.rocketmq.streams.common.configurable.IConfigurable; +import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.MessageOffset; import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; @@ -36,18 +37,18 @@ public interface ISink extends IConfigurable, IStageBuilder, /** * 根据channel推断 meta,或者不需要meta,如消息对垒 * - * @param fieldName2Value + * @param message * @return */ - boolean batchAdd(IMessage fieldName2Value, ISplit split); + boolean batchAdd(IMessage message, ISplit split); /** * 根据channel推断 meta,或者不需要meta,如消息对垒 * - * @param fieldName2Value + * @param context * @return */ - boolean batchAdd(IMessage fieldName2Value); + boolean batchAdd(IMessage message); /** * 直接存储存储,不过缓存 diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/AbstractMultiSplitMessageCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/AbstractMultiSplitMessageCache.java index 837b58cd..136ab6d5 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/AbstractMultiSplitMessageCache.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/sinkcache/impl/AbstractMultiSplitMessageCache.java @@ -29,6 +29,7 @@ import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache; import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.utils.StringUtil; public abstract class AbstractMultiSplitMessageCache extends MessageCache { protected ConcurrentHashMap> queueMessageCaches = new ConcurrentHashMap(); @@ -38,9 +39,9 @@ public abstract class AbstractMultiSplitMessageCache extends MessageCache public AbstractMultiSplitMessageCache( IMessageFlushCallBack flushCallBack) { super(null); - this.executorService = new ThreadPoolExecutor(10, 10, - 0L, TimeUnit.MILLISECONDS, - new LinkedBlockingQueue()); +// this.executorService = new ThreadPoolExecutor(10, 10, +// 0L, TimeUnit.MILLISECONDS, +// new LinkedBlockingQueue()); this.flushCallBack = new MessageFlushCallBack(flushCallBack); } @@ -52,7 +53,7 @@ public int addCache(R msg) { synchronized (this) { messageCache = queueMessageCaches.get(queueId); if (messageCache == null) { - messageCache = new MessageCache(flushCallBack); + messageCache = createMessageCache(); messageCache.setAutoFlushSize(this.autoFlushSize); messageCache.setAutoFlushTimeGap(this.autoFlushTimeGap); messageCache.setBatchSize(batchSize); @@ -77,6 +78,10 @@ public int addCache(R msg) { return size; } + protected MessageCache createMessageCache(){ + return new MessageCache(flushCallBack); + } + protected abstract String createSplitId(R msg); @Override @@ -98,7 +103,11 @@ public int flush(Set splitIds) { return 0; } if (splitIds.size() == 1) { - IMessageCache cache = queueMessageCaches.get(splitIds.iterator().next()); + String spiltId = splitIds.iterator().next(); + if(StringUtil.isEmpty(spiltId)){ + return 0; + } + IMessageCache cache = queueMessageCaches.get(spiltId); if (cache == null) { return 0; } @@ -108,6 +117,9 @@ public int flush(Set splitIds) { } CountDownLatch countDownLatch = new CountDownLatch(splitIds.size()); for (String splitId : splitIds) { + if(StringUtil.isEmpty(splitId)){ + continue; + } executorService.execute(new Runnable() { @Override public void run() { IMessageCache cache = queueMessageCaches.get(splitId); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSource.java index 939b1d54..2bc7e55f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSource.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSource.java @@ -21,6 +21,8 @@ import java.io.UnsupportedEncodingException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -58,12 +60,20 @@ public abstract class AbstractSource extends BasedConfigurable implements ISource, ILifeCycle { public static String CHARSET = "UTF-8"; + /** + * 输入的消息是否为json + */ + protected Boolean isJsonData = true; + /** + * 输入的消息是否为json array + */ + protected Boolean msgIsJsonArray = false; - protected Boolean isJsonData = true;//输入的消息是否为json - protected Boolean msgIsJsonArray = false;//输入的消息是否为json array @ENVDependence - protected String groupName;//group name + protected String groupName; + protected int maxThread = Runtime.getRuntime().availableProcessors(); + @ENVDependence protected String topic = ""; protected String namesrvAddr; @@ -79,13 +89,30 @@ public abstract class AbstractSource extends BasedConfigurable implements ISourc * 每次拉取的最大条数,多用于消息队列 */ protected int maxFetchLogGroupSize = 100; - protected List logFingerprintFields;//log fingerprint to filter msg quickly - protected String encoding = CHARSET;//字节编码方式 - protected String fieldDelimiter;//如果是分割符分割,分割符 - protected MetaData metaData;//主要用于分割符拆分字段当场景 + /** + * log fingerprint to filter msg quickly + */ + protected List logFingerprintFields; + /** + * 字节编码方式 + */ + protected String encoding = CHARSET; + /** + * 如果是分割符分割,分割符 + */ + protected String fieldDelimiter; + /** + * 主要用于分割符拆分字段当场景 + */ + protected MetaData metaData; + protected List headerFieldNames; + /** + * if set the value,the data will be shuffled to a new topic + */ + protected int shuffleConcurrentCount; /** * 数据源投递消息的算子,此算子用来接收source的数据,做处理 */ @@ -100,12 +127,13 @@ public abstract class AbstractSource extends BasedConfigurable implements ISourc /** * 做checkpoint的管理 */ - protected transient CheckPointManager checkPointManager = new CheckPointManager(); + protected transient CheckPointManager checkPointManager = null; @Override protected boolean initConfigurable() { hasStart = new AtomicBoolean(false); openMock = false; + checkPointManager = new CheckPointManager(); return super.initConfigurable(); } @@ -119,6 +147,12 @@ public boolean start(IStreamOperator receiver) { return isStartSucess; } + @Override public void destroy() { + if (hasStart.compareAndSet(true, false)) { + super.destroy(); + } + } + /** * 启动 source * @@ -139,8 +173,7 @@ public AbstractSource() { public AbstractContext doReceiveMessage(JSONObject message, boolean needSetCheckPoint, String queueId, String offset) { Message msg = createMessage(message, queueId, offset, needSetCheckPoint); - AbstractContext context = executeMessage(msg); - return context; + return executeMessage(msg); } /** @@ -261,23 +294,23 @@ public JSONObject create(String message) { if (this.fieldDelimiter != null) { String[] values = message.split(this.fieldDelimiter); - List fields = this.metaData.getMetaDataFields(); + List> fields = this.metaData.getMetaDataFields(); if (values.length != this.metaData.getMetaDataFields().size()) { throw new RuntimeException("expect table column's count equals data size (" + fields.size() + "," + values.length + ")"); } for (int i = 0; i < values.length; i++) { - MetaDataField field = fields.get(i); - String fildName = field.getFieldName(); + MetaDataField field = fields.get(i); + String fieldName = field.getFieldName(); String valueStr = values[i]; Object value = field.getDataType().getData(valueStr); - msg.put(fildName, value); + msg.put(fieldName, value); } return msg; } else { //单字段场景 - List metaDataFields = this.metaData.getMetaDataFields(); - MetaDataField metaDataField = null; - for (MetaDataField field : metaDataFields) { + List> metaDataFields = this.metaData.getMetaDataFields(); + MetaDataField metaDataField = null; + for (MetaDataField field : metaDataFields) { if (this.headerFieldNames == null) { metaDataField = field; break; @@ -292,19 +325,17 @@ public JSONObject create(String message) { return msg; } } + return msg; } else { //sdk场景 if (this.fieldDelimiter != null) { String[] values = message.split(this.fieldDelimiter); - List columns = new ArrayList<>(); - for (String value : values) { - columns.add(value); - } - return createJson(columns); + return createJson(Arrays.asList(values)); + } else { + return createJson(message); } } - return createJson(message); } /** @@ -664,7 +695,7 @@ public boolean isBatchMessage() { @Override public String createCheckPointName() { - ISource source = this; + ISource source = this; String namespace = source.getNameSpace(); String name = source.getConfigureName(); @@ -689,6 +720,14 @@ public String createCheckPointName() { } + public int getShuffleConcurrentCount() { + return shuffleConcurrentCount; + } + + public void setShuffleConcurrentCount(int shuffleConcurrentCount) { + this.shuffleConcurrentCount = shuffleConcurrentCount; + } + @Override public boolean isFinished() { return false; @@ -730,4 +769,6 @@ public List getHeaderFieldNames() { public void setHeaderFieldNames(List headerFieldNames) { this.headerFieldNames = headerFieldNames; } + + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractUnreliableSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractUnreliableSource.java index 608f108e..4eab84f6 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractUnreliableSource.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractUnreliableSource.java @@ -40,7 +40,7 @@ public abstract class AbstractUnreliableSource extends AbstractBatchSource { private static final Log LOG = LogFactory.getLog(AbstractUnreliableSource.class); - protected Boolean enableAsyncReceive = true; + protected Boolean enableAsyncReceive = false; protected boolean isSingleType = false;//是否只有单个生产者,如果是,则为true private transient ExecutorService cachedThreadPool = null; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/ISplit.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/ISplit.java index 27886c65..a1432c08 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/ISplit.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/split/ISplit.java @@ -29,14 +29,11 @@ public interface ISplit extends Comparable, Serializable, IJsonable { String getQueueId(); - /** * 获取具体的队列 获取具体的队列 * - * @return + * @return Q */ Q getQueue(); - // public T getQueue(); - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/ComponentCreator.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/ComponentCreator.java index a0ec41fb..3efac351 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/ComponentCreator.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/component/ComponentCreator.java @@ -82,6 +82,10 @@ public static void setProperties(Properties properties) { ComponentCreator.properties = properties; } + public static void updateProperties(Properties properties) { + ComponentCreator.properties.putAll(properties); + } + public static String[] createKV(Properties properties) { List keys = new ArrayList<>(); Iterator keyIterator = properties.keySet().iterator(); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/AbstractConfigurable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/AbstractConfigurable.java index 9152da2b..bfd76008 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/AbstractConfigurable.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/AbstractConfigurable.java @@ -40,8 +40,6 @@ public abstract class AbstractConfigurable extends Entity implements IConfigurab protected transient IConfigurableService configurableService; - protected long updateFlag = 0;//通过它来触发更新,其他字段变更都不会触发更新 - /** * 是否完成初始化 */ @@ -117,14 +115,14 @@ public static String createSQL(IConfigurable configurable) { public static String createSQL(IConfigurable configurable, String tableName) { String json = configurable.toJson(); Entity entity = null; - if (Entity.class.isInstance(configurable)) { + if (configurable instanceof Entity) { entity = (Entity) configurable; } else { entity = new Entity(); } int status = 1; if (configurable.getPrivateData("status") != null) { - status = Integer.valueOf(configurable.getPrivateData("status")); + status = Integer.parseInt(configurable.getPrivateData("status")); } String theSecretValue; try { @@ -178,10 +176,6 @@ public Map getPrivateData() { return this.privateDatas; } - public Map getPrivateDatas() { - return privateDatas; - } - public void setPrivateDatas(Map privateDatas) { this.privateDatas = privateDatas; } @@ -202,15 +196,8 @@ public boolean isHasInit() { return hasInit; } - public long getUpdateFlag() { - return updateFlag; - } - public void setHasInit(boolean hasInit) { this.hasInit = hasInit; } - public void setUpdateFlag(long updateFlag) { - this.updateFlag = updateFlag; - } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/BasedConfigurable.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/BasedConfigurable.java index a501e950..180e9914 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/BasedConfigurable.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/BasedConfigurable.java @@ -48,6 +48,8 @@ public class BasedConfigurable extends AbstractConfigurable { protected String version = "1.0"; + protected long updateFlag = 0;//通过它来触发更新,其他字段变更都不会触发更新 + @Override public String getNameSpace() { return nameSpace; @@ -163,7 +165,7 @@ protected void getJsonObject(Class clazz, JSONObject jsonObject) { } DataType dataType = DataTypeUtil.createFieldDataType(this, field.getName()); String fieldJsonStr = jsonObject.getString(field.getName()); - fieldJsonStr = getENVParamter(field, fieldJsonStr); + fieldJsonStr = getENVParameter(field, fieldJsonStr); Object fieldValue = dataType.getData(fieldJsonStr); if (fieldValue != null) { ReflectUtil.setBeanFieldValue(this, field.getName(), fieldValue); @@ -181,7 +183,7 @@ protected void getJsonObject(Class clazz, JSONObject jsonObject) { * @param fieldValue * @return */ - protected String getENVParamter(Field field, String fieldValue) { + protected String getENVParameter(Field field, String fieldValue) { ENVDependence dependence = field.getAnnotation(ENVDependence.class); if (dependence == null) { return fieldValue; @@ -249,4 +251,11 @@ public void setVersion(String version) { this.version = version; } + public long getUpdateFlag() { + return updateFlag; + } + + public void setUpdateFlag(long updateFlag) { + this.updateFlag = updateFlag; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableService.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableService.java index 13b7d120..55a4791f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableService.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configurable/IConfigurableService.java @@ -45,9 +45,12 @@ public interface IConfigurableService { String FILE_SERVICE_NAME = "file"; + String FILE_PATH_NAME = "filePathAndName"; + String HTTP_SERVICE_NAME = "http"; - String FILE_PATH_NAME = "filePathAndName"; + String HTTP_SERVICE_ENDPOINT = "dipper.configurable.service.type.http.endpoint"; + /** * 启动定时任务,定期从存储加载对象到内存 diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/ConfigureFileKey.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/ConfigureFileKey.java index 96306a01..0f7fcc58 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/ConfigureFileKey.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/configure/ConfigureFileKey.java @@ -148,7 +148,6 @@ static String getDipperJdbcPassword() { String WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX = "window.shuffle.channel."; String WINDOW_SYSTEM_MESSAGE_CHENNEL_OWNER = "window.system.message.channel.owner";//如果能做消息过滤,只过滤本window的消息,可以配置这个属性,如rocketmq的tags.不支持的会做客户端过滤 - /** * 通知相关 */ @@ -163,4 +162,10 @@ static String getDipperJdbcPassword() { String IS_ATOMIC_DB_SINK = "isAtomicDbSink"; + /** + * minibatch + */ + + String WINDOW_MINIBATCH_SWITCH="window.minibatch.switch";//是否启动minibatch + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/AbstractContext.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/AbstractContext.java index 48e3ee6a..2ead3cd6 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/AbstractContext.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/AbstractContext.java @@ -30,6 +30,7 @@ import org.apache.rocketmq.streams.common.monitor.MonitorFactory; import org.apache.rocketmq.streams.common.optimization.FilterResultCache; import org.apache.rocketmq.streams.common.optimization.HomologousVar; +import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; public abstract class AbstractContext extends HashMap { @@ -68,6 +69,9 @@ public abstract class AbstractContext extends HashMap { protected Map homologousResult; + //未触发规则的表达式 + protected List notFireExpressionMonitor=new ArrayList<>(); + public AbstractContext(T message) { this.message = message; } @@ -96,6 +100,8 @@ public > void syncContext(C subContext) { this.isBreak = subContext.isBreak; this.quickFilterResult = subContext.quickFilterResult; this.homologousResult = subContext.homologousResult; + this.isContinue=subContext.isContinue; + this.notFireExpressionMonitor=subContext.notFireExpressionMonitor; } public > C syncSubContext(C subContext) { @@ -109,6 +115,8 @@ public > C syncSubContext(C subContext) { subContext.isBreak = isBreak; subContext.quickFilterResult = quickFilterResult; subContext.homologousResult = homologousResult; + subContext.isContinue=isContinue; + subContext.notFireExpressionMonitor=notFireExpressionMonitor; return subContext; } @@ -137,8 +145,7 @@ public Boolean matchFromHomologousCache(IMessage message, HomologousVar var) { if (bitSet == null) { return null; } - boolean result = bitSet.get(var.getIndex()); - return result; + return bitSet.get(var.getIndex()); } public void resetIsContinue() { @@ -409,6 +416,19 @@ public IMonitor createChildrenMonitor(String parentMintorName, IConfigurable con return monitor.createChildren(configurable); } + public NotFireReason getNotFireReason(){ + return (NotFireReason)this.get("NotFireReason"); + } + + + public void setNotFireReason(NotFireReason notFireReason){ + this.put("NotFireReason",notFireReason); + } + + + public void removeNotFireReason(){ + this.remove("NotFireReason"); + } public boolean isBreak() { return isBreak; } @@ -430,7 +450,16 @@ public void setHomologousResult( this.homologousResult = homologousResult; } + public List getNotFireExpressionMonitor() { + return notFireExpressionMonitor; + } + + public void setNotFireExpressionMonitor(List notFireExpressionMonitor) { + this.notFireExpressionMonitor = notFireExpressionMonitor; + } + public FilterResultCache getQuickFilterResult() { return quickFilterResult; } + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageHeader.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageHeader.java index d9561cd5..0b105430 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageHeader.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/context/MessageHeader.java @@ -17,6 +17,7 @@ package org.apache.rocketmq.streams.common.context; import com.alibaba.fastjson.JSONObject; +import java.util.HashSet; import java.util.Set; import org.apache.rocketmq.streams.common.channel.source.ISource; import org.apache.rocketmq.streams.common.channel.split.ISplit; @@ -38,7 +39,7 @@ public class MessageHeader { * 因为是字符串比较,需要有一个固定位数 */ public static final int SPLIT_OFFST_INIT = 10000000; - protected String piplineName; + protected String pipelineName; /** * 当前消息的channel信息 @@ -47,11 +48,11 @@ public class MessageHeader { /** * 路由用到的路由标签,标签的值是stage的label,用于路由stage,可以多个label */ - private String routeLables; + private String routeLabels; /** * 路由用到的路由标签,标签的值是stage的label,用于路由stage,主要用于过滤,可以多个label */ - private String filterLables; + private String filterLabels; /** * 消息所属的queue id */ @@ -86,15 +87,10 @@ public class MessageHeader { */ private int loopIndex = -1; - /** - * 规则不被触发对应的表达式 - */ - protected TopologyFilterMonitor piplineExecutorMonitor; - /** * 在pipline中消息会被拆分,在有多分支时,会被copy,这个对象会在任何变动时,都保持全局唯一,不允许copy,复制,创建,一个message全局唯一 */ - protected MessageGlobleTrace messageGloableTrace; + protected MessageGlobleTrace messageGlobalTrace; /** * trace id of every message @@ -108,20 +104,19 @@ public class MessageHeader { public MessageHeader copy() { MessageHeader header = new MessageHeader(); header.setSource(source); - header.routeLables = routeLables; - header.filterLables = filterLables; + header.routeLabels = routeLabels; + header.filterLabels = filterLabels; header.queueId = queueId; header.messageOffset = new MessageOffset(messageOffset.getOffsetStr(), messageOffset.isLongOfMainOffset()); header.sendTime = sendTime; header.needFlush = needFlush; header.isSystemMessage = isSystemMessage; header.progress = new BatchMessageOffset(); - header.piplineExecutorMonitor = piplineExecutorMonitor; if (progress != null) { header.progress.setCurrentMessage(progress.getCurrentMessage()); header.progress.setOwnerType(progress.getOwnerType()); } - header.messageGloableTrace = messageGloableTrace;//这里不必复制,会保持全局唯一 + header.messageGlobalTrace = messageGlobalTrace;//这里不必复制,会保持全局唯一 header.traceId = traceId; header.msgRouteFromLable = msgRouteFromLable; header.logFingerprintValue = logFingerprintValue; @@ -135,21 +130,33 @@ public JSONObject toJsonObject() { ReflectUtil.setFieldValue2Object(this, jsonObject); return jsonObject; } - + public Set createRouteLableSet(String routeLabels){ + if(routeLabels==null){ + return null; + } + String[] lables=MapKeyUtil.splitKey(routeLabels); + Set routeLableSet=new HashSet<>(); + for(String lable:lables){ + routeLableSet.add(lable); + } + return routeLableSet; + } /** - * 用于路由的标签,标签等于stage的lable + * 用于路由的标签,标签等于stage的label * * @param labels */ - public String addRouteLable(String... labels) { - return createLables(routeLables, labels); + public String addRouteLabel(String... labels) { + this.routeLabels = createLables(routeLabels, labels); + return this.routeLabels; } /** - * 用于路由的标签,标签等于stage的lable + * 用于路由的标签,标签等于stage的label */ - public String addFilterLable(String... labels) { - return createLables(filterLables, labels); + public String addFilterLabel(String... labels) { + this.filterLabels = createLables(filterLabels, labels); + return this.filterLabels; } public void setQueueId(String queueId) { @@ -164,12 +171,13 @@ public void setSource(ISource source) { this.source = source; } - public String getRouteLables() { - return routeLables; + public String getRouteLabels() { + return routeLabels; } - public String getFilterLables() { - return filterLables; + + public String getFilterLabels() { + return filterLabels; } public String getQueueId() { @@ -230,6 +238,8 @@ protected String createLables(String routeLabels, String... labels) { return routeLabels; } + + public boolean isNeedFlush() { return needFlush; } @@ -238,12 +248,12 @@ public void setNeedFlush(boolean needFlush) { this.needFlush = needFlush; } - public void setRouteLables(String routeLables) { - this.routeLables = routeLables; + public void setRouteLabels(String routeLabels) { + this.routeLabels = routeLabels; } - public void setFilterLables(String filterLables) { - this.filterLables = filterLables; + public void setFilterLabels(String filterLabels) { + this.filterLabels = filterLabels; } public void setSendTime(long sendTime) { @@ -302,21 +312,14 @@ public void setSystemMessage(boolean systemMessage) { isSystemMessage = systemMessage; } - public TopologyFilterMonitor getPiplineExecutorMonitor() { - return piplineExecutorMonitor; - } - public void setPiplineExecutorMonitor( - TopologyFilterMonitor piplineExecutorMonitor) { - this.piplineExecutorMonitor = piplineExecutorMonitor; - } - public MessageGlobleTrace getMessageGloableTrace() { - return messageGloableTrace; + public MessageGlobleTrace getMessageGlobalTrace() { + return messageGlobalTrace; } - public void setMessageGloableTrace(MessageGlobleTrace messageGloableTrace) { - this.messageGloableTrace = messageGloableTrace; + public void setMessageGlobalTrace(MessageGlobleTrace messageGlobalTrace) { + this.messageGlobalTrace = messageGlobalTrace; } public String getMsgRouteFromLable() { @@ -349,11 +352,11 @@ public void setOffset(Long offset) { messageOffset.isLongOfMainOffset = true; } - public String getPiplineName() { - return piplineName; + public String getPipelineName() { + return pipelineName; } - public void setPiplineName(String piplineName) { - this.piplineName = piplineName; + public void setPipelineName(String pipelineName) { + this.pipelineName = pipelineName; } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IntDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IntDataType.java index 572114f8..87a3f464 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IntDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/IntDataType.java @@ -36,7 +36,8 @@ public Integer getData(String jsonValue) { if (jsonValue == null || "N/A".equals(jsonValue)) { return null; } - return Integer.valueOf(jsonValue); + + return Double.valueOf(jsonValue).intValue(); } @Override diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ShortDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ShortDataType.java index 88940303..63a3391c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ShortDataType.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/ShortDataType.java @@ -47,7 +47,7 @@ public Short getData(String jsonValue) { if (jsonValue == null || "N/A".equals(jsonValue)) { return null; } - return Short.valueOf(jsonValue); + return Double.valueOf(jsonValue).shortValue(); } @Override diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISerialize.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISerialize.java new file mode 100644 index 00000000..a675f382 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/interfaces/ISerialize.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.common.interfaces; + +public interface ISerialize { + + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreator.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreator.java index a6572afb..39a4838c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreator.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreator.java @@ -20,12 +20,11 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -import org.apache.rocketmq.streams.common.utils.NameCreatorUtil; public class NameCreator { - private transient AtomicInteger nameCreator = new AtomicInteger(10000); + private transient AtomicInteger nameCreator = new AtomicInteger(100000000); - private static Map creatorMap = new HashMap<>(); + private Map creatorMap = new HashMap<>(); /** * 每个规则一个名字生成器,expression name @@ -33,13 +32,13 @@ public class NameCreator { * @param names * @return */ - public static NameCreator createOrGet(String... names) { + public NameCreator createOrGet(String... names) { String ruleName = MapKeyUtil.createKeyBySign("_", names); NameCreator nameCreator = creatorMap.get(ruleName); if (nameCreator != null) { return nameCreator; } - synchronized (NameCreatorUtil.class) { + synchronized (NameCreator.class) { nameCreator = creatorMap.get(ruleName); if (nameCreator != null) { return nameCreator; @@ -50,13 +49,12 @@ public static NameCreator createOrGet(String... names) { return nameCreator; } - public static String createNewName(String... names) { + public String createNewName(String... names) { NameCreator nameCreator = createOrGet(names); return nameCreator.createName(names); } public String createName(String... namePrefix) { - String value = MapKeyUtil.createKeyBySign("_", MapKeyUtil.createKeyBySign("_", namePrefix), nameCreator.incrementAndGet() + ""); - return value; + return MapKeyUtil.createKeyBySign("_", MapKeyUtil.createKeyBySign("_", namePrefix), nameCreator.incrementAndGet() + ""); } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreatorContext.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreatorContext.java new file mode 100644 index 00000000..ba21be0e --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/model/NameCreatorContext.java @@ -0,0 +1,44 @@ +/* + * 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.rocketmq.streams.common.model; + +public class NameCreatorContext { + protected static ThreadLocal threadLocal = new ThreadLocal<>(); + + private NameCreatorContext(NameCreator nameCreator) { + threadLocal.set(nameCreator); + } + + public static NameCreator get() { + + NameCreator nameCreator = threadLocal.get(); + if (nameCreator == null) { + nameCreator = new NameCreator(); + threadLocal.set(nameCreator); + } + return nameCreator; + } + + public static void remove() { + threadLocal.remove(); + } + + public static NameCreatorContext init(NameCreator nameCreator) { + return new NameCreatorContext(nameCreator); + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/ConsoleMonitorManager.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/ConsoleMonitorManager.java new file mode 100644 index 00000000..a3837ae0 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/ConsoleMonitorManager.java @@ -0,0 +1,412 @@ +/* + * 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.rocketmq.streams.common.monitor; + +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.channel.IChannel; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.monitor.model.JobStage; +import org.apache.rocketmq.streams.common.monitor.model.TraceIdsDO; +import org.apache.rocketmq.streams.common.monitor.model.TraceMonitorDO; +import org.apache.rocketmq.streams.common.monitor.service.MonitorDataSyncService; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.model.AbstractStage; + +public class ConsoleMonitorManager { + + private static final Log LOG = LogFactory.getLog(ConsoleMonitorManager.class); + + public static final int MSG_FILTERED = -1; + public static final int MSG_NOT_FLOWED = 0; + public static final int MSG_FLOWED = 1; + + private static ConsoleMonitorManager monitorManager = new ConsoleMonitorManager(); + private Map cache = new ConcurrentHashMap(); + private Map traceCache = new ConcurrentHashMap(); + private Set validTraceIds = new HashSet(); + private MonitorDataSyncService monitorDataSyncService = MonitorDataSyncServiceFactory.create(); + + public static ConsoleMonitorManager getInstance() { + return monitorManager; + } + + /** + * 上面使用 static 定义并初始化了一个 ConsoleMonitorManager 实例,所以当这类被加载时,就会执行构造方法 + * 构造方法会开启一个定时任务,任务会定时执行一个线程,该线程动作如下: + * 1. 查询出所有有效的 traceId 保存下来 + * 2. 更新 dipper_job_stage 表 + * 3. 更新 dipper_trace_monitor 表 + */ + public ConsoleMonitorManager() { + if (!isConsoleOpen()) { + return; + } + + ScheduledExecutorService executorService = Executors.newScheduledThreadPool(1); + executorService.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + try { + queryValidTraceIds(); + Map jobStageMap = cache; + cache = new ConcurrentHashMap(); + long current = System.currentTimeMillis(); + for (JobStage jobStage : jobStageMap.values()) { + jobStage.setMachineName(""); +// jobStage.setLastInputMsgObj(new JSONObject()); +// String msg = msgObj.toJSONString(); +// if (msg != null && !"".equalsIgnoreCase(msg)){ +// jobStage.setLastInputMsg(msg); +// } + jobStage.setInput(jobStage.getSafeInput().getAndSet(0)); + jobStage.setOutput(jobStage.getSafeOutput().getAndSet(0)); + + double tps = jobStage.getInput() / ((current - jobStage.getCreateTime()) * 1.0 / 1000); + jobStage.setTps((double) (Math.round(tps * 100)) / 100); + + } + monitorDataSyncService.updateJobStage(jobStageMap.values()); +// for (JobStage jobStage : jobStageMap.values()) { +// jobStage.setMachineName(""); +// String msg = JSON.toJSONString(jobStage.getLastInputMsgObj()); +// if (msg != null && msg.length() > 2){ +// jobStage.setLastInputMsg(msg); +// } +// jobStage.setInput(jobStage.getSafeInput().getAndSet(0)); +// jobStage.setOutput(jobStage.getSafeOutput().getAndSet(0)); +// if(jobStage.getInput() != 0 || jobStage.getOutput() != 0 || jobStage.getPrevInput() != 0){ +// monitorDataSyncService.updateJobStage(jobStage); +//// DBManager.executeSQL("update dipper_job_stage set input = input+#{input},prev_input = prev_input+#{prevInput},output = output+#{output},last_input_msg=#{lastInputMsg},last_input_msg_time=#{lastInputMsgTime},last_output_msg_time=#{lastOutputMsgTime} where stage_name = #{stageName}",jobStage); +// } +// jobStage.setPrevInput(jobStage.getInput()); +// } + if (validTraceIds.size() > 0) { + for (TraceMonitorDO traceMonitorDO : traceCache.values()) { + traceMonitorDO.setInputNumber(traceMonitorDO.getSafeInput().getAndSet(0)); + traceMonitorDO.setOutputNumber(traceMonitorDO.getSafeOutput().getAndSet(0)); + if (traceMonitorDO.getInputNumber() != 0 || traceMonitorDO.getOutputNumber() != 0) { + monitorDataSyncService.addTraceMonitor(traceMonitorDO); + } + } + } + } catch (Exception e) { + LOG.error("ConsoleMonitorManager report error!", e); + } + } + }, 20, 30, TimeUnit.SECONDS); + } + + public Set getValidTraceIds() { + return validTraceIds; + } + + /** + * traceId 记录 + */ + public void reportChannel(ChainPipeline pipeline, ISource source, IMessage message) { + if (!isConsoleOpen()) { + return; + } + + long clientTime = message.getHeader().getSendTime(); + JSONObject msg = message.getMessageBody(); + //目前pipeline只支持一个channel 暂时写死channel name + JobStage jobStage = getJobStage(source.getConfigureName() + "_source_0"); + +// // 必须是开启了 traceId,才能对 dipper_trace_monitor 表中的数据进行修改 +// // 只是消息中带有 traceId 不行 +// if (validTraceIds.contains(message.getHeader().getTraceId())){ +// // 查出来当前的任务名称 +// List jobNameList = DBManager.querySQL("select job_name from dipper_job_stage where stage_name =" + "'" + jobStage.getStageName() + "'", String.class); +// if (!jobNameList.isEmpty()){ +// String jobName = JSONObject.parseObject(jobNameList.get(0), HashMap.class).get("job_name").toString(); +// String key = jobName + "_" + message.getHeader().getTraceId(); +// +// // 如果 任务+traceId 不在 dipper_trace_monitor 中 +// if (!jobMonitorCache.containsKey(key)){ +// jobMonitorCache.put(key, ""); +// }else{ +// // 如果 任务+traceId 在 dipper_trace_monitor 中,则将表中对应记录删除掉 +// // 避免上一次的消息对本次产生影响 +// TraceMonitorDO monitorDO = new TraceMonitorDO(); +// monitorDO.setTraceId(message.getHeader().getTraceId()); +// monitorDO.setJobName(jobName); +// // 直接删除掉其实也行 +// DBManager.executeSQL("delete from dipper_trace_monitor where trace_id=#{traceId} and job_name=#{jobName}", monitorDO); +// } +// } +// } + + jobStage.getSafeInput().incrementAndGet(); + jobStage.setLastInputMsgObj(msg); + if (clientTime != 0) { + jobStage.setLastInputMsgTime(new Date(clientTime)); + } else { + jobStage.setLastInputMsgTime(new Date()); + } + jobStage.getSafeOutput().incrementAndGet(); + jobStage.setLastOutputMsgTime(new Date()); + + String traceId = message.getHeader().getTraceId(); + if (validTraceIds.contains(traceId)) { + if (!message.getHeader().isSystemMessage()) { + // 记录 traceId + // getTraceMonitor() 会从 traceCache 中取 traceMonitor,没有会新建一个并添加到 traceCache 中 + TraceMonitorDO traceMonitor = getTraceMonitor(source.getConfigureName() + "_source_0", traceId); + // 表示消息正常流转过了此 stage + traceMonitor.setStatus(1); + traceMonitor.getSafeInput().incrementAndGet(); + traceMonitor.getSafeOutput().incrementAndGet(); + traceMonitor.setLastInputMsgTime(new Date()); + traceMonitor.setLastOutputMsgTime(new Date()); + traceMonitor.setInputLastMsg(msg.toJSONString()); + traceMonitor.setOutputLastMsg(msg.toJSONString()); + traceMonitor.setJobName(pipeline.getConfigureName()); + } + } + + } + + /** + * 输入 traceId 记录 + */ + public void reportInput(AbstractStage stage, IMessage message) { + if (!isConsoleOpen()) { + return; + } + + JSONObject msg = message.getMessageBody(); + JobStage jobStage = getJobStage(stage.getLabel()); + jobStage.getSafeInput().incrementAndGet(); + jobStage.setLastInputMsgObj(msg); + jobStage.setLastInputMsg(msg.toJSONString()); + jobStage.setLastInputMsgTime(new Date()); + + String traceId = message.getHeader().getTraceId(); + String shuffleTraceId = msg.getString("SHUFFLE_TRACE_ID"); + if (validTraceIds.contains(traceId) || (shuffleTraceId != null && shuffleTraceId.contains(traceId))) { + // getTraceMonitor() 会从 traceCache 中取 traceMonitor,没有会新建一个并添加到 traceCache 中 + TraceMonitorDO traceMonitor = getTraceMonitor(stage.getLabel(), traceId); + traceMonitor.getSafeInput().incrementAndGet(); + traceMonitor.setInputLastMsg(msg.toJSONString()); + traceMonitor.setLastInputMsgTime(new Date()); + traceMonitor.setJobName(stage.getPipeline().getConfigureName()); + } + } + + /** + * 输出 traceId 记录 + * status: + * 0 未流转到此 stage + * -1 消息在此 stage 被过滤掉 + * 1 未发生异常 + */ + public void reportOutput(AbstractStage stage, IMessage message, int status, String exceptionMsg) { + if (!isConsoleOpen()) { + return; + } + + JSONObject msg = message.getMessageBody(); + // 从 cache 中获取 jobStage + JobStage jobStage = getJobStage(stage.getLabel()); + if (status == MSG_FLOWED) { + jobStage.getSafeOutput().incrementAndGet(); + jobStage.setLastOutputMsgTime(new Date()); + } + String traceId = message.getHeader().getTraceId(); + String shuffleTraceId = msg.getString("SHUFFLE_TRACE_ID"); + + // 如果配置了有效的 traceId 就把监控信息存储起来 + if (validTraceIds.contains(traceId) || (shuffleTraceId != null && shuffleTraceId.contains(traceId))) { + // getTraceMonitor() 会从 traceCache 中取 traceMonitor,没有会新建一个并添加到 traceCache 中 + TraceMonitorDO traceMonitor = getTraceMonitor(stage.getLabel(), traceId); + // 因为 status 可能会被覆盖掉,所以这里面判断一下 status 有没有赋值,如果没有赋值再赋值 + traceMonitor.setStatus(status); + if (status == MSG_FILTERED) { + traceMonitor.setExceptionMsg(exceptionMsg); + } else if (status == MSG_FLOWED) { + traceMonitor.getSafeOutput().incrementAndGet(); + traceMonitor.setOutputLastMsg(msg.toJSONString()); + traceMonitor.setLastOutputMsgTime(new Date()); + } + traceMonitor.setJobName(stage.getPipeline().getConfigureName()); + } + } + + public JobStage getJobStage(String uniqKey) { +// String key = createKey(uniqKey); + JobStage jobStage = cache.get(uniqKey); + if (jobStage == null) { + synchronized (uniqKey) { + if (cache.get(uniqKey) == null) { + jobStage = new JobStage(); + jobStage.setStageName(uniqKey); + cache.put(uniqKey, jobStage); + } + } + } + return jobStage; + } + + public TraceMonitorDO getTraceMonitor(String uniqKey, String traceId) { + String key = createKey(uniqKey, traceId); + TraceMonitorDO traceMonitor = traceCache.get(key); + if (traceMonitor == null) { + synchronized (key) { + if (traceCache.get(key) == null) { + traceMonitor = new TraceMonitorDO(); + traceMonitor.setStageName(uniqKey); + traceMonitor.setTraceId(traceId); + traceCache.put(key, traceMonitor); + } + } + } + return traceMonitor; + } + + public String createKey(String... uniqKeys) { + //通过线程名称实现线程隔离 + StringBuffer sb = new StringBuffer(Thread.currentThread().getName()); + for (String key : uniqKeys) { + sb.append(key); + } + return sb.toString(); + } + + public void queryValidTraceIds() { + List traceIdsDOS = monitorDataSyncService.getTraceIds(); + if (traceIdsDOS != null && traceIdsDOS.size() > 0) { + validTraceIds.clear(); + for (TraceIdsDO traceIdsDO : traceIdsDOS) { + validTraceIds.add(traceIdsDO.getTraceId()); + } + } + + } + + private boolean isConsoleOpen() { + String configurableServiceType = ComponentCreator.getProperties().getProperty(DataSyncConstants.UPDATE_TYPE); + if (DataSyncConstants.UPDATE_TYPE_ROCKETMQ.equalsIgnoreCase(configurableServiceType) || + DataSyncConstants.UPDATE_TYPE_HTTP.equalsIgnoreCase(configurableServiceType) || + DataSyncConstants.UPDATE_TYPE_DB.equalsIgnoreCase(configurableServiceType)) { + return true; + } + return false; + } + +// public void saveJobName(List traceMonitorDOS){ +// if (traceMonitorDOS==null || traceMonitorDOS.size()==0){ +// return; +// } +// for (TraceMonitorDO traceMonitorDO : traceMonitorDOS){ +// // 添加到集合中 +// String key = traceMonitorDO.getStageName(); +// if (jobNameCache.containsKey(key)){ +// jobNameCache.get(key).value = traceMonitorDO.getJobName(); +// moveToHead(jobNameCache.get(key)); +// }else{ +// if (jobNameCache.size() > MAX_SIZE){ +// deleteNode(); +// } +// // 无论是否删除,都要把节点插入到首个结点 +// ListNode node = new ListNode(key, traceMonitorDO.getJobName()); +// jobNameCache.put(key, node); +// head.next.prev = node; +// node.next = head.next; +// node.prev = head; +// head.next = node; +// } +// } +// } +// +// public String getJobName(String stageName){ +// if (!jobNameCache.containsKey(stageName)){ +// return null; +// } +// moveToHead(jobNameCache.get(stageName)); +// return jobNameCache.get(stageName).value; +// } +// +// public void moveToHead(ListNode node){ +// if (node.next != null){ +// node.prev.next = node.next; +// node.next.prev = node.prev; +// }else{ +// node.prev.next = null; +// } +// node.next = head.next; +// head.next.prev = node; +// head.next = node; +// node.prev = head; +// } +// +// public void deleteNode(){ +// jobNameCache.remove(tail.prev.key); +// tail.prev.prev.next = tail; +// tail.prev = tail.prev.prev; +// } +// +// public void initJobCache(){ +// jobNameCache = new HashMap<>(); +// head = new ListNode(); +// tail = new ListNode(); +// head.next = tail; +// } + + class ListNode { + public String key; + public String value; + public ListNode prev; + public ListNode next; + + public ListNode(String key, String value) { + this.key = key; + this.value = value; + } + + public ListNode() { + + } + } + + public static void main(String[] args) { + long a = 6l; + System.out.println((float) (a / ((10000 - 79) / 1000))); + System.out.println((float) (a / 16)); + System.out.println((10000 - 79) * 1.0 / 1000); +// Math. + } + +} + + diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/DataSyncConstants.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/DataSyncConstants.java new file mode 100644 index 00000000..612e15a4 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/DataSyncConstants.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.common.monitor; + +public class DataSyncConstants { + + /** + * rocketmq-stream更新模块对应的topic + */ + public static final String RULE_UP_TOPIC = "dipper.console.topic.up"; + + /** + * rocketmq-stream更新模块对应的topic + */ + public static final String RULE_UP_TAG = "dipper.console.topic.up.tags"; + + /** + * rocketmq-stream更新模块对应的topic + */ + public static final String RULE_DOWN_TOPIC = "dipper.console.topic.down"; + + /** + * rocketmq-stream更新模块对应的topic + */ + public static final String RULE_DOWN_TAG = "dipper.console.topic.down.tags"; + + /** + * rocketmq-stream更新模块对应的tag + */ + public static final String RULE_TOPIC_TAG = "dipper.console.tag"; + + /** + * rocketmq-stream更新模块对应的消息渠道类型 默认为metaq + */ + public static final String UPDATE_TYPE = "dipper.console.service.type"; + + public static final String UPDATE_TYPE_HTTP = "http"; + public static final String UPDATE_TYPE_DB = "db"; + public static final String UPDATE_TYPE_ROCKETMQ = "rocketmq"; +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/HttpClient.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/HttpClient.java new file mode 100644 index 00000000..e0dd7411 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/HttpClient.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.common.monitor; + +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSession; +import org.apache.http.Consts; +import org.apache.http.Header; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.config.ConnectionConfig; +import org.apache.http.config.Registry; +import org.apache.http.config.RegistryBuilder; +import org.apache.http.conn.socket.ConnectionSocketFactory; +import org.apache.http.conn.socket.PlainConnectionSocketFactory; +import org.apache.http.conn.ssl.SSLConnectionSocketFactory; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; +import org.apache.http.ssl.SSLContexts; +import org.apache.http.ssl.TrustStrategy; + +public class HttpClient { + + public static final int NORMAL_STATUES = 200; + public static final String CHARSET = "UTF-8"; + public static final int TIMOUT = 10000; + public static final int CONNECT_TIMOUT = 10000; + + protected String accessId; + protected String accessIdSecret; + protected String endPoint; + + protected transient CloseableHttpClient client; + + public HttpClient(String accessId, String accessIdSecret, String endPoint) { + this.accessId = accessId; + this.accessIdSecret = accessIdSecret; + this.endPoint = endPoint; + init(); + } + + public void init() { + RequestConfig.Builder configBuilder = RequestConfig.custom(); + configBuilder.setConnectionRequestTimeout(CONNECT_TIMOUT); + configBuilder.setConnectTimeout(CONNECT_TIMOUT); + configBuilder.setSocketTimeout(TIMOUT); + SSLConnectionSocketFactory sslsf = null; + try { + SSLContext sslcontext = SSLContexts.custom().loadTrustMaterial(new TrustStrategy() { + @Override + public boolean isTrusted(X509Certificate[] chain, String authType) throws CertificateException { + return true; + } + }).build(); + sslsf = new SSLConnectionSocketFactory(sslcontext, new HostnameVerifier() { + @Override + public boolean verify(String s, SSLSession sslSession) { + return true; + } + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + ConnectionConfig connectionConfig = ConnectionConfig.custom().setCharset(Consts.UTF_8).build(); + Registry socketFactoryRegistry = RegistryBuilder.create().register("https", + sslsf).register("http", + new PlainConnectionSocketFactory()).build(); + PoolingHttpClientConnectionManager connManager = new PoolingHttpClientConnectionManager(socketFactoryRegistry); + connManager.setDefaultConnectionConfig(connectionConfig); + connManager.setMaxTotal(500); + connManager.setDefaultMaxPerRoute(50); + HttpClientBuilder clientBuilder = HttpClients.custom(); + + clientBuilder.setDefaultRequestConfig(configBuilder.build()); + clientBuilder.setSSLSocketFactory(sslsf); + clientBuilder.setConnectionManager(connManager); + + client = clientBuilder.build(); + } + + public CloseableHttpResponse get(String url, Header... headers) { + try { + HttpGet httpGet = new HttpGet(url); + if (headers != null && headers.length > 0) { + for (Header header : headers) { + httpGet.addHeader(header); + } + } + CloseableHttpResponse response = client.execute(httpGet); + return response; + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/HttpUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/HttpUtil.java new file mode 100644 index 00000000..b5c789ae --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/HttpUtil.java @@ -0,0 +1,248 @@ +/* + * 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.rocketmq.streams.common.monitor; + +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSession; +import org.apache.commons.lang3.StringUtils; +import org.apache.http.Consts; +import org.apache.http.Header; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpDelete; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPatch; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.config.ConnectionConfig; +import org.apache.http.config.Registry; +import org.apache.http.config.RegistryBuilder; +import org.apache.http.conn.socket.ConnectionSocketFactory; +import org.apache.http.conn.socket.PlainConnectionSocketFactory; +import org.apache.http.conn.ssl.SSLConnectionSocketFactory; +import org.apache.http.conn.ssl.TrustStrategy; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; +import org.apache.http.ssl.SSLContexts; +import org.apache.http.util.EntityUtils; + +public class HttpUtil { + + public static final int NORMAL_STATUES = 200; + public static final String CHARSET = "UTF-8"; + public static final int TIMOUT = 10000; + public static final int CONNECT_TIMOUT = 10000; + + protected String accessId; + protected String accessIdSecret; + protected String endPoint; + + private static CloseableHttpClient httpclient; + + public HttpUtil(String accessId, String accessIdSecret, String endPoint) { + this.accessId = accessId; + this.accessIdSecret = accessIdSecret; + this.endPoint = endPoint; + init(); + } + + private void init() { + RequestConfig.Builder configBuilder = RequestConfig.custom(); + configBuilder.setConnectionRequestTimeout(CONNECT_TIMOUT); + configBuilder.setConnectTimeout(CONNECT_TIMOUT); + configBuilder.setSocketTimeout(TIMOUT); + SSLConnectionSocketFactory sslsf = null; + try { + SSLContext sslcontext = SSLContexts.custom().loadTrustMaterial(new TrustStrategy() { + @Override + public boolean isTrusted(X509Certificate[] chain, String authType) throws CertificateException { + return true; + } + }).build(); + sslsf = new SSLConnectionSocketFactory(sslcontext, new HostnameVerifier() { + @Override + public boolean verify(String s, SSLSession sslSession) { + return true; + } + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + ConnectionConfig connectionConfig = ConnectionConfig.custom().setCharset(Consts.UTF_8).build(); + Registry socketFactoryRegistry = RegistryBuilder.create().register("https", sslsf).register("http", new PlainConnectionSocketFactory()).build(); + PoolingHttpClientConnectionManager connManager = new PoolingHttpClientConnectionManager(socketFactoryRegistry); + connManager.setDefaultConnectionConfig(connectionConfig); + connManager.setMaxTotal(500); + connManager.setDefaultMaxPerRoute(50); + HttpClientBuilder clientBuilder = HttpClients.custom(); + + clientBuilder.setDefaultRequestConfig(configBuilder.build()); + clientBuilder.setSSLSocketFactory(sslsf); + clientBuilder.setConnectionManager(connManager); + + httpclient = clientBuilder.build(); + } + + public static String getContent(String url) { + return getContent(url, null); + } + + public static String getContent(String url, Header... headers) { + try { + return EntityUtils.toString(get(url, headers).getEntity(), CHARSET); + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } + + public static String deleteContent(String url) { + return deleteContent(url, null); + } + + public static String deleteContent(String url, Header... headers) { + try { + return EntityUtils.toString(delete(url, headers).getEntity(), CHARSET); + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } + + public static String postContent(String url, String body) { + return postContent(url, body, null); + } + + public static String patchContent(String url, String body) { + return patchContent(url, body, null); + } + + public static String postContent(String url, String body, Header... headers) { + try { + return EntityUtils.toString(post(url, body, headers).getEntity(), CHARSET); + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } + + public static String patchContent(String url, String body, Header... headers) { + try { + return EntityUtils.toString(patch(url, body, headers).getEntity(), CHARSET); + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } + + public static CloseableHttpResponse get(String url) { + return get(url); + } + + public static CloseableHttpResponse delete(String url) { + return delete(url); + } + + public static CloseableHttpResponse get(String url, Header... headers) { + try { + HttpGet httpGet = new HttpGet(url); + if (headers != null && headers.length > 0) { + for (Header header : headers) { + httpGet.addHeader(header); + } + } + return httpclient.execute(httpGet); + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } + + public static CloseableHttpResponse delete(String url, Header... headers) { + try { + HttpDelete httpDelete = new HttpDelete(url); + if (headers != null && headers.length > 0) { + for (Header header : headers) { + httpDelete.addHeader(header); + } + } + return httpclient.execute(httpDelete); + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } + + public static CloseableHttpResponse post(String url, String body) { + return post(url, body); + } + + public static CloseableHttpResponse patch(String url, String body) { + return patch(url, body, null); + } + + public static CloseableHttpResponse post(String url, String body, Header... headers) { + try { + HttpPost httpPost = new HttpPost(url); + StringEntity stringEntity = new StringEntity(body, CHARSET); + stringEntity.setContentEncoding(CHARSET); + stringEntity.setContentType("application/json"); + httpPost.setEntity(stringEntity); + if (headers != null && headers.length > 0) { + for (Header header : headers) { + httpPost.addHeader(header); + } + } + return httpclient.execute(httpPost); + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } + + public static CloseableHttpResponse patch(String url, String body, Header... headers) { + try { + HttpPatch httpPatch = new HttpPatch(url); + StringEntity stringEntity = new StringEntity(body, CHARSET); + stringEntity.setContentEncoding(CHARSET); + stringEntity.setContentType("application/json"); + httpPatch.setEntity(stringEntity); + if (headers != null && headers.length > 0) { + for (Header header : headers) { + httpPatch.addHeader(header); + } + } + return httpclient.execute(httpPatch); + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } + + public static String replaceSpace(String str) { + if (StringUtils.isNotBlank(str)) { + return str.replace(" ", "").replace("\r", "").replace("\n", "").replace("\r\n", ""); + } + return str; + } +} + diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/MonitorDataSyncServiceFactory.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/MonitorDataSyncServiceFactory.java new file mode 100644 index 00000000..cf8125b6 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/MonitorDataSyncServiceFactory.java @@ -0,0 +1,61 @@ +/* + * 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.rocketmq.streams.common.monitor; + +import org.apache.rocketmq.streams.common.component.AbstractComponent; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.monitor.service.MonitorDataSyncService; +import org.apache.rocketmq.streams.common.monitor.service.impl.DBMonitorDataSyncImpl; +import org.apache.rocketmq.streams.common.monitor.service.impl.HttpMonitorDataSyncImpl; +import org.apache.rocketmq.streams.common.monitor.service.impl.RocketMQMonitorDataSyncImpl; + +public class MonitorDataSyncServiceFactory { + + public static MonitorDataSyncService create() { + String configureType = ComponentCreator.getProperties().getProperty(DataSyncConstants.UPDATE_TYPE); + if (DataSyncConstants.UPDATE_TYPE_DB.equalsIgnoreCase(configureType)) { + return new DBMonitorDataSyncImpl(); + } else if (DataSyncConstants.UPDATE_TYPE_HTTP.equalsIgnoreCase(configureType)) { + String accessId = ComponentCreator.getProperties().getProperty(AbstractComponent.HTTP_AK); + String accessIdSecret = ComponentCreator.getProperties().getProperty(AbstractComponent.HTTP_SK); + String endPoint = ComponentCreator.getProperties().getProperty(IConfigurableService.HTTP_SERVICE_ENDPOINT); + return new HttpMonitorDataSyncImpl(accessId, accessIdSecret, endPoint); + } else if (DataSyncConstants.UPDATE_TYPE_ROCKETMQ.equalsIgnoreCase(configureType)) { + return new RocketMQMonitorDataSyncImpl(); + } + +// try { +// Properties properties1 = new Properties(); +// properties1.putAll(properties); +// String type = properties1.getProperty(CONFIGURABLE_SERVICE_TYPE); +// if (StringUtil.isEmpty(type)) { +// type = IConfigurableService.DEFAULT_SERVICE_NAME; +// ; +// } +// IConfigurableService configurableService = getConfigurableServcieType(type); +// if (AbstractSupportParentConfigureService.class.isInstance(configurableService)) { +// ((AbstractSupportParentConfigureService)configurableService).initMethod(properties1); +// } +// } catch (Exception e) { +// LOG.error("create ConfigurableService error", e); +// return null; +// } + return null; + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/group/MonitorCommander.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/group/MonitorCommander.java index 87d47625..63d7ea70 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/group/MonitorCommander.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/group/MonitorCommander.java @@ -117,11 +117,11 @@ public void init(ISink... outputDataSource) { result.put("errorCount", gmi.getErrorCount()); result.put("slowCount", gmi.getSlowCount()); //本地打印 - loggerOutputDataSource.batchAdd(new Message(result)); + loggerOutputDataSource.batchAdd(new Message(result),null); loggerOutputDataSource.flush(); //远程输出 for (ISink source : outputDataSourceList) { - source.batchAdd(new Message(result)); + source.batchAdd(new Message(result),null); } } //flush出去 diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/DipperMonitor.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/DipperMonitor.java index 72c77b6e..f63286c6 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/DipperMonitor.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/impl/DipperMonitor.java @@ -418,7 +418,7 @@ protected void output2Channel(String level) { } try { if (channel != null) { - channel.batchAdd(new Message(result)); + channel.batchAdd(new Message(result),null); } } catch (Exception e) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/JobStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/JobStage.java new file mode 100644 index 00000000..ebb61924 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/JobStage.java @@ -0,0 +1,350 @@ +/* + * 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.rocketmq.streams.common.monitor.model; + +import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson.annotation.JSONField; +import java.util.Date; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Description: + * + * @author 苏同亮 + * Date 2021-05-14 + */ +public class JobStage { + + /** + * 主键 + */ + private int id; + + /** + * 任务名称 + */ + private String jobName; + + /** + * 机器名称 + */ + private String machineName; + + /** + * stage名称 + */ + private String stageName; + + /** + * stage类型 + */ + private String stageType; + + /** + * 输入数据量 + */ + private long input; + private long prevInput; + private AtomicInteger safeInput = new AtomicInteger(); + + /** + * 输出数据量 + */ + private long output; + private AtomicInteger safeOutput = new AtomicInteger(); + + /** + * 消息处理速度 + */ + private double tps; + + /** + * 最后输入数据 + */ + private String lastInputMsg; + private JSONObject lastInputMsgObj; + + /** + * 最后输入数据时间 + */ + @JSONField(format = "yyyy-MM-dd HH:mm:ss") + private Date lastInputMsgTime; + + /** + * 最后输出数据时间 + */ + @JSONField(format = "yyyy-MM-dd HH:mm:ss") + private Date lastOutputMsgTime; + + /** + * stage下游集合 + */ + private String nextStageLables; + + /** + * stage上游集合 + */ + private String prevStageLables; + + /** + * stage内容 + */ + private String stageContent; + + @JSONField(format = "yyyy-MM-dd HH:mm:ss") + private Date pingTime; + + private long createTime = System.currentTimeMillis(); + + /** + * setter for column 主键 + */ + public void setId(int id) { + this.id = id; + } + + /** + * getter for column 主键 + */ + public int getId() { + return this.id; + } + + /** + * setter for column 任务名称 + */ + public void setJobName(String jobName) { + this.jobName = jobName; + } + + /** + * getter for column 任务名称 + */ + public String getJobName() { + return this.jobName; + } + + /** + * setter for column 机器名称 + */ + public void setMachineName(String machineName) { + this.machineName = machineName; + } + + /** + * getter for column 机器名称 + */ + public String getMachineName() { + return this.machineName; + } + + /** + * setter for column stage名称 + */ + public void setStageName(String stageName) { + this.stageName = stageName; + } + + /** + * getter for column stage名称 + */ + public String getStageName() { + return this.stageName; + } + + /** + * setter for column stage类型 + */ + public void setStageType(String stageType) { + this.stageType = stageType; + } + + /** + * getter for column stage类型 + */ + public String getStageType() { + return this.stageType; + } + + /** + * setter for column 输入数据量 + */ + public void setInput(long input) { + this.input = input; + } + + /** + * getter for column 输入数据量 + */ + public long getInput() { + return this.input; + } + + /** + * setter for column 输出数据量 + */ + public void setOutput(long output) { + this.output = output; + } + + /** + * getter for column 输出数据量 + */ + public long getOutput() { + return this.output; + } + + public double getTps() { + return tps; + } + + public void setTps(double tps) { + this.tps = tps; + } + + /** + * setter for column 最后输入数据 + */ + public void setLastInputMsg(String lastInputMsg) { + this.lastInputMsg = lastInputMsg; + } + + /** + * getter for column 最后输入数据 + */ + public String getLastInputMsg() { + return this.lastInputMsg; + } + + /** + * setter for column 最后输入数据时间 + */ + public void setLastInputMsgTime(Date lastInputMsgTime) { + this.lastInputMsgTime = lastInputMsgTime; + } + + /** + * getter for column 最后输入数据时间 + */ + public Date getLastInputMsgTime() { + return this.lastInputMsgTime; + } + + /** + * setter for column 最后输出数据时间 + */ + public void setLastOutputMsgTime(Date lastOutputMsgTime) { + this.lastOutputMsgTime = lastOutputMsgTime; + } + + /** + * getter for column 最后输出数据时间 + */ + public Date getLastOutputMsgTime() { + return this.lastOutputMsgTime; + } + + /** + * setter for column stage下游集合 + */ + public void setNextStageLables(String nextStageLables) { + this.nextStageLables = nextStageLables; + } + + /** + * getter for column stage下游集合 + */ + public String getNextStageLables() { + return this.nextStageLables; + } + + /** + * setter for column stage上游集合 + */ + public void setPrevStageLables(String prevStageLables) { + this.prevStageLables = prevStageLables; + } + + /** + * getter for column stage上游集合 + */ + public String getPrevStageLables() { + return this.prevStageLables; + } + + /** + * setter for column stage内容 + */ + public void setStageContent(String stageContent) { + this.stageContent = stageContent; + } + + /** + * getter for column stage内容 + */ + public String getStageContent() { + return this.stageContent; + } + + public Date getPingTime() { + return pingTime; + } + + public void setPingTime(Date pingTime) { + this.pingTime = pingTime; + } + + public long getCreateTime() { + return createTime; + } + + public void setCreateTime(long createTime) { + this.createTime = createTime; + } + + public JSONObject getLastInputMsgObj() { + return lastInputMsgObj; + } + + public void setLastInputMsgObj(JSONObject lastInputMsgObj) { + this.lastInputMsgObj = lastInputMsgObj; + } + + public AtomicInteger getSafeInput() { + return safeInput; + } + + public void setSafeInput(AtomicInteger safeInput) { + this.safeInput = safeInput; + } + + public AtomicInteger getSafeOutput() { + return safeOutput; + } + + public void setSafeOutput(AtomicInteger safeOutput) { + this.safeOutput = safeOutput; + } + + public long getPrevInput() { + return prevInput; + } + + public void setPrevInput(long prevInput) { + this.prevInput = prevInput; + } +} \ No newline at end of file diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceIdsDO.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceIdsDO.java new file mode 100644 index 00000000..a205cde9 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceIdsDO.java @@ -0,0 +1,126 @@ +/* + * 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.rocketmq.streams.common.monitor.model; + +import com.alibaba.fastjson.annotation.JSONField; +import java.util.Date; + +/** + * Description: + * + * @author 苏同亮 + * Date 2021-06-21 + */ +public class TraceIdsDO { + + /** + * 主键 + */ + private int id; + + /** + * tarceid + */ + private String traceId; + + /** + * 过期时间 + */ + @JSONField(format = "yyyy-MM-dd HH:mm:ss") + private Date gmtExpire; + + /** + * 创建时间 + */ + @JSONField(format = "yyyy-MM-dd HH:mm:ss") + private Date gmtCreate; + + private String useStatus; + private String jobName; + + /** + * setter for column 主键 + */ + public void setId(int id) { + this.id = id; + } + + /** + * getter for column 主键 + */ + public int getId() { + return this.id; + } + + /** + * setter for column tarceid + */ + public void setTraceId(String traceId) { + this.traceId = traceId; + } + + /** + * getter for column tarceid + */ + public String getTraceId() { + return this.traceId; + } + + /** + * setter for column 过期时间 + */ + public void setGmtExpire(Date gmtExpire) { + this.gmtExpire = gmtExpire; + } + + /** + * getter for column 过期时间 + */ + public Date getGmtExpire() { + return this.gmtExpire; + } + + /** + * setter for column 创建时间 + */ + public void setGmtCreate(Date gmtCreate) { + this.gmtCreate = gmtCreate; + } + + /** + * getter for column 创建时间 + */ + public Date getGmtCreate() { + return this.gmtCreate; + } + + public String getUseStatus() { + return useStatus; + } + + public void setUseStatus(String useStatus) { + this.useStatus = useStatus; + } + + public String getJobName() { + return jobName; + } + + public void setJobName(String jobName) { + this.jobName = jobName; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceMonitorDO.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceMonitorDO.java new file mode 100644 index 00000000..5f278883 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/model/TraceMonitorDO.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.rocketmq.streams.common.monitor.model; + +import com.alibaba.fastjson.annotation.JSONField; +import java.util.Date; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Description: + * + * @author 苏同亮 + * Date 2021-06-21 + */ +public class TraceMonitorDO { + + /** + * 主键 + */ + private int id; + + /** + * traceid + */ + private String traceId; + + /** + * stagename + */ + private String stageName; + + /** + * 输入数量 + */ + private int inputNumber; + private AtomicInteger safeInput = new AtomicInteger(); + /** + * 输入最后一条消息 + */ + private String inputLastMsg; + + /** + * 输出数量 + */ + private int outputNumber; + private AtomicInteger safeOutput = new AtomicInteger(); + + /** + * 输出最后一条消息 + */ + private String outputLastMsg; + + /** + * 最后一条输入时间 + */ + @JSONField(format = "yyyy-MM-dd HH:mm:ss") + private Date lastInputMsgTime; + + /** + * 最后一条输出时间 + */ + @JSONField(format = "yyyy-MM-dd HH:mm:ss") + private Date lastOutputMsgTime; + + /** + * 当前 stage 的消息流转状态 + * -1 消息在此 stage 被过滤掉 + * 0 消息为流转到此 stage + * 1 正常流转 + */ + private Integer status; + + /** + * 异常信息 + */ + private String exceptionMsg; + + /** + * 此 traceId 对应的 jobName + */ + private String jobName; + + /** + * setter for column 主键 + */ + public void setId(int id) { + this.id = id; + } + + /** + * getter for column 主键 + */ + public int getId() { + return this.id; + } + + /** + * setter for column traceid + */ + public void setTraceId(String traceId) { + this.traceId = traceId; + } + + /** + * getter for column traceid + */ + public String getTraceId() { + return this.traceId; + } + + /** + * setter for column stagename + */ + public void setStageName(String stageName) { + this.stageName = stageName; + } + + /** + * getter for column stagename + */ + public String getStageName() { + return this.stageName; + } + + /** + * setter for column 输入数量 + */ + public void setInputNumber(int inputNumber) { + this.inputNumber = inputNumber; + } + + /** + * getter for column 输入数量 + */ + public int getInputNumber() { + return this.inputNumber; + } + + /** + * setter for column 输入最后一条消息 + */ + public void setInputLastMsg(String inputLastMsg) { + this.inputLastMsg = inputLastMsg; + } + + /** + * getter for column 输入最后一条消息 + */ + public String getInputLastMsg() { + return this.inputLastMsg; + } + + /** + * setter for column 输出数量 + */ + public void setOutputNumber(int outputNumber) { + this.outputNumber = outputNumber; + } + + /** + * getter for column 输出数量 + */ + public int getOutputNumber() { + return this.outputNumber; + } + + /** + * setter for column 输出最后一条消息 + */ + public void setOutputLastMsg(String outputLastMsg) { + this.outputLastMsg = outputLastMsg; + } + + /** + * getter for column 输出最后一条消息 + */ + public String getOutputLastMsg() { + return this.outputLastMsg; + } + + public Date getLastInputMsgTime() { + return lastInputMsgTime; + } + + public void setLastInputMsgTime(Date lastInputMsgTime) { + this.lastInputMsgTime = lastInputMsgTime; + } + + public Date getLastOutputMsgTime() { + return lastOutputMsgTime; + } + + public void setLastOutputMsgTime(Date lastOutputMsgTime) { + this.lastOutputMsgTime = lastOutputMsgTime; + } + + public AtomicInteger getSafeInput() { + return safeInput; + } + + public void setSafeInput(AtomicInteger safeInput) { + this.safeInput = safeInput; + } + + public AtomicInteger getSafeOutput() { + return safeOutput; + } + + public void setSafeOutput(AtomicInteger safeOutput) { + this.safeOutput = safeOutput; + } + + public String getJobName() { + return jobName; + } + + public void setJobName(String jobName) { + this.jobName = jobName; + } + + public String getExceptionMsg() { + return exceptionMsg; + } + + public void setExceptionMsg(String exceptionMsg) { + this.exceptionMsg = exceptionMsg; + } + + public Integer getStatus() { + return status; + } + + public void setStatus(Integer status) { + this.status = status; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/MonitorDataSyncService.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/MonitorDataSyncService.java new file mode 100644 index 00000000..423aabdd --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/MonitorDataSyncService.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.common.monitor.service; + +import java.util.Collection; +import java.util.List; +import org.apache.rocketmq.streams.common.monitor.model.JobStage; +import org.apache.rocketmq.streams.common.monitor.model.TraceIdsDO; +import org.apache.rocketmq.streams.common.monitor.model.TraceMonitorDO; + +public interface MonitorDataSyncService { + + List getTraceIds(); + + void updateJobStage(Collection jobStages); + + void addTraceMonitor(TraceMonitorDO traceMonitorDO); +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/DBMonitorDataSyncImpl.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/DBMonitorDataSyncImpl.java new file mode 100644 index 00000000..7512521c --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/DBMonitorDataSyncImpl.java @@ -0,0 +1,63 @@ +/* + * 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.rocketmq.streams.common.monitor.service.impl; + +import java.util.Collection; +import java.util.List; +import org.apache.rocketmq.streams.common.monitor.model.JobStage; +import org.apache.rocketmq.streams.common.monitor.model.TraceIdsDO; +import org.apache.rocketmq.streams.common.monitor.model.TraceMonitorDO; +import org.apache.rocketmq.streams.common.monitor.service.MonitorDataSyncService; + +public class DBMonitorDataSyncImpl implements MonitorDataSyncService { + @Override public List getTraceIds() { + return null; + } + + @Override public void updateJobStage(Collection jobStages) { + + } + + @Override public void addTraceMonitor(TraceMonitorDO traceMonitorDO) { + + } +// @Override +// public List getTraceIds() { +// +// List traceIdsDOS = ORMUtil.queryForList("SELECT * FROM dipper_trace_ids WHERE now() < gmt_expire", null, TraceIdsDO.class); +//// ORMUtil.querySQL("SELECT * FROM dipper_trace_ids WHERE now() < gmt_expire", TraceIdsDO.class); +// return traceIdsDOS; +// } +// +// @Override +// public void updateJobStage(JobStage jobStage) { +// ORMUtil.executeSQL("update dipper_job_stage set input = input+#{input},prev_input = prev_input+#{prevInput}," + +// "output = output+#{output},last_input_msg=#{lastInputMsg},last_input_msg_time=#{lastInputMsgTime}," + +// "last_output_msg_time=#{lastOutputMsgTime} where stage_name = #{stageName}", jobStage); +// } +// +// @Override +// public void addTraceMonitor(TraceMonitorDO traceMonitorDO) { +// ORMUtil.executeSQL("insert into dipper_trace_monitor(trace_id,stage_name,input_number,input_last_msg," + +// "output_number,output_last_msg,last_input_msg_time,last_output_msg_time) " + +// "values(#{traceId},#{stageName},#{inputNumber},#{inputLastMsg},#{outputNumber},#{outputLastMsg}," + +// "#{lastInputMsgTime},#{lastOutputMsgTime}) ON DUPLICATE KEY UPDATE input_number = input_number+#{inputNumber}," + +// "output_number = output_number+#{outputNumber},input_last_msg=#{inputLastMsg},output_last_msg=#{outputLastMsg}," + +// "last_input_msg_time=#{lastInputMsgTime},last_output_msg_time=#{lastOutputMsgTime}",traceMonitorDO); +// +// } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/HttpMonitorDataSyncImpl.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/HttpMonitorDataSyncImpl.java new file mode 100644 index 00000000..b0bf8d6a --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/HttpMonitorDataSyncImpl.java @@ -0,0 +1,151 @@ +/* + * 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.rocketmq.streams.common.monitor.service.impl; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; +import java.io.UnsupportedEncodingException; +import java.nio.charset.Charset; +import java.nio.charset.UnsupportedCharsetException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.http.HttpEntity; +import org.apache.http.ParseException; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.entity.ContentType; +import org.apache.http.protocol.HTTP; +import org.apache.http.util.Args; +import org.apache.http.util.CharArrayBuffer; +import org.apache.rocketmq.streams.common.monitor.HttpUtil; +import org.apache.rocketmq.streams.common.monitor.model.JobStage; +import org.apache.rocketmq.streams.common.monitor.model.TraceIdsDO; +import org.apache.rocketmq.streams.common.monitor.model.TraceMonitorDO; +import org.apache.rocketmq.streams.common.monitor.service.MonitorDataSyncService; + +public class HttpMonitorDataSyncImpl implements MonitorDataSyncService { + + protected String endPoint; + + public static final String GET_TRACE_IDS = "/queryValidTraceId"; + + public static final String UPDATE_JOBSTAGE = "/updateJobStage"; + + public static final String ADD_TRACEMONITOR = "/insertTraceMonitor"; + + protected HttpUtil client; + + public HttpMonitorDataSyncImpl(String accessId, String accessIdSecret, String endPoint) { + client = new HttpUtil(accessId, accessIdSecret, endPoint); + this.endPoint = endPoint; + } + + @Override + public List getTraceIds() { + List traceIdsDOS = new ArrayList<>(); + CloseableHttpResponse response = client.get(endPoint + GET_TRACE_IDS, null); + if (response != null && response.getStatusLine().getStatusCode() == 200) { + traceIdsDOS = convert(response, TraceIdsDO.class); + } + + return traceIdsDOS; + } + + @Override + public void updateJobStage(Collection jobStages) { + + String response = client.postContent(endPoint + UPDATE_JOBSTAGE, JSONObject.toJSONString(jobStages)); + + } + + @Override + public void addTraceMonitor(TraceMonitorDO traceMonitorDO) { + String response = client.postContent(endPoint + ADD_TRACEMONITOR, JSONObject.toJSONString(traceMonitorDO)); + } + + private List convert(CloseableHttpResponse response, Class clazz) { + try { + String content = toString(response.getEntity(), Charset.forName("UTF-8")); + JSONObject object = JSONObject.parseObject(content); + String data = object.getString("data"); + List result = JSONArray.parseArray(data, clazz); + return result; + + } catch (IOException e) { + e.printStackTrace(); + } + return null; + } + + public String toString(HttpEntity entity, Charset defaultCharset) throws IOException, ParseException { + Args.notNull(entity, "Entity"); + InputStream instream = entity.getContent(); + if (instream == null) { + return null; + } else { + try { + Args.check(entity.getContentLength() <= 2147483647L, "HTTP entity too large to be buffered in memory"); + int i = (int) entity.getContentLength(); + if (i < 0) { + i = 4096; + } + + Charset charset = null; + + try { + ContentType contentType = ContentType.get(entity); + if (contentType != null) { + charset = contentType.getCharset(); + } + } catch (UnsupportedCharsetException var13) { + if (defaultCharset == null) { + throw new UnsupportedEncodingException(var13.getMessage()); + } + } + + if (charset == null) { + charset = defaultCharset; + } + + if (charset == null) { + charset = HTTP.DEF_CONTENT_CHARSET; + } + + Reader reader = new InputStreamReader(instream, charset); + CharArrayBuffer buffer = new CharArrayBuffer(i); + char[] tmp = new char[1024]; + + int l; + while ((l = reader.read(tmp)) != -1) { + buffer.append(tmp, 0, l); + } + + String var9 = buffer.toString(); + return var9; + } finally { + instream.close(); + } + } + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/RocketMQMonitorDataSyncImpl.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/RocketMQMonitorDataSyncImpl.java new file mode 100644 index 00000000..95707ab5 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/monitor/service/impl/RocketMQMonitorDataSyncImpl.java @@ -0,0 +1,185 @@ +/* + * 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.rocketmq.streams.common.monitor.service.impl; + +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly; +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.monitor.DataSyncConstants; +import org.apache.rocketmq.streams.common.monitor.model.JobStage; +import org.apache.rocketmq.streams.common.monitor.model.TraceIdsDO; +import org.apache.rocketmq.streams.common.monitor.model.TraceMonitorDO; +import org.apache.rocketmq.streams.common.monitor.service.MonitorDataSyncService; +import org.apache.rocketmq.streams.common.utils.IPUtil; +import org.apache.rocketmq.streams.common.utils.RuntimeUtil; + +public class RocketMQMonitorDataSyncImpl implements MonitorDataSyncService { + + protected final Log LOG = LogFactory.getLog(RocketMQMonitorDataSyncImpl.class); + + public static final String QUERYVALIDTRACEID = "queryValidTraceId"; + public static final String UPDATEJOBSTAGE = "updateJobStage"; + public static final String INSERTTRACEMONITOR = "insertTraceMonitor"; + + private Updater updater; + + private List traceIdsDOS; + + public RocketMQMonitorDataSyncImpl() { + updater = new Updater(); + updater.init(); + } + + @Override + public List getTraceIds() { + return traceIdsDOS; + } + + @Override + public void updateJobStage(Collection jobStages) { + if (jobStages.isEmpty()) { + return; + } + JSONObject object = new JSONObject(); + object.put("operate", UPDATEJOBSTAGE); + object.put("data", jobStages); + updater.sendMsg(object); + + } + + @Override + public void addTraceMonitor(TraceMonitorDO traceMonitorDO) { + JSONObject object = new JSONObject(); + object.put("operate", INSERTTRACEMONITOR); + object.put("data", traceMonitorDO); + updater.sendMsg(object); + + } + + public String updateTraceIds(JSONObject object) { + List ids = new ArrayList<>(); + JSONArray array = object.getJSONArray("data"); + if (array != null && array.size() > 0) { + for (int i = 0; i < array.size(); i++) { + ids.add(array.getObject(i, TraceIdsDO.class)); + } + } + this.traceIdsDOS = ids; + + return Updater.RESULT_SUCCESS; + } + + private String dealMessage(String message) { + JSONObject object = JSON.parseObject(message); + String operate = object.getString("operate"); +// Long operateTime = object.getLong("operate_time"); + if (QUERYVALIDTRACEID.equalsIgnoreCase(operate)) { + return updateTraceIds(object); + } + return Updater.RESULT_SUCCESS; + } + + class Updater { + public static final String RESULT_SUCCESS = "success"; + public static final String RESULT_FAILED = "failed"; + + protected Long pullIntervalMs; + protected String ruleUpTopic = ComponentCreator.getProperties().getProperty(DataSyncConstants.RULE_UP_TOPIC); + protected String ruleUpTag = ComponentCreator.getProperties().getProperty(DataSyncConstants.RULE_UP_TAG, "T_MSG_DIPPER_RULE"); + protected String ruleDownTopic = ComponentCreator.getProperties().getProperty(DataSyncConstants.RULE_DOWN_TOPIC); + protected String ruleDownTag = ComponentCreator.getProperties().getProperty(DataSyncConstants.RULE_DOWN_TAG, "T_MSG_DIPPER_RULE_PUSH"); + public String CHARSET = "UTF-8"; + + protected transient DefaultMQProducer producer; + + public void init() { + initConsumer(); + initProducer(); + + } + + protected DefaultMQPushConsumer initConsumer() { + try { +// DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(IPUtil.getLocalIdentification().replaceAll("\\.", "_")); + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("jobconfig_comsumer" + RuntimeUtil.getPid()); + if (pullIntervalMs != null) { + consumer.setPullInterval(pullIntervalMs); + } +// consumer.setNamesrvAddr(this.namesrvAddr); + consumer.subscribe(ruleDownTopic, ruleDownTag); + consumer.registerMessageListener((MessageListenerOrderly) (msgs, context) -> { + try { + int i = 0; + for (MessageExt msg : msgs) { + String ruleMsg = new String(msg.getBody(), CHARSET); + LOG.info("receive message is :" + ruleMsg); + dealMessage(ruleMsg); + } + } catch (Exception e) { + LOG.error("consume message from rocketmq error " + e, e); + e.printStackTrace(); + } + return ConsumeOrderlyStatus.SUCCESS;// 返回消费成功 + }); + + consumer.start(); + return consumer; + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("start metaq channel error " + ruleUpTopic, e); + } + } + + protected void initProducer() { + producer = new DefaultMQProducer(RuntimeUtil.getDipperInstanceId() + "producer"); + try { + producer.start(); + } catch (Exception e) { + LOG.error("init producer error " + e, e); + throw new RuntimeException("init producer error, msg=" + e.getMessage(), e); + } + } + + protected void sendMsg(JSONObject msg) { + sendMsg(msg, ruleUpTopic); + } + + protected void sendMsg(JSONObject msg, String topic) { + try { + LOG.info("sendMsg is: " + msg.toJSONString() + " topic is: " + topic + " tag is: " + ruleUpTag); + Message message = new Message(topic, ruleUpTag, null, msg.toJSONString().getBytes("UTF-8")); + producer.send(message); + } catch (Exception e) { + LOG.error("updater sendMsg error: ", e); + e.printStackTrace(); + } + } + + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousOptimization.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousOptimization.java index a5f69576..ffbaf502 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousOptimization.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/IHomologousOptimization.java @@ -23,7 +23,7 @@ public interface IHomologousOptimization { - void optimizate(List pipelineList, int cacheSize, int preFingerprintCacheSize); + void optimizate(List> pipelineList, int cacheSize, int preFingerprintCacheSize); void calculate(IMessage message, AbstractContext context); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/MessageGlobleTrace.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/MessageGlobleTrace.java index f90ebc4e..f33a7a2c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/MessageGlobleTrace.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/MessageGlobleTrace.java @@ -35,7 +35,7 @@ private MessageGlobleTrace() { * @return */ public static Boolean existFinishBranch(IMessage message) { - MessageGlobleTrace trace = message.getHeader().getMessageGloableTrace(); + MessageGlobleTrace trace = message.getHeader().getMessageGlobalTrace(); if (trace == null) { return null; } @@ -48,7 +48,7 @@ public static Boolean existFinishBranch(IMessage message) { * @param message */ public static void finishPipeline(IMessage message) { - MessageGlobleTrace trace = message.getHeader().getMessageGloableTrace(); + MessageGlobleTrace trace = message.getHeader().getMessageGlobalTrace(); if (trace == null) { return; } @@ -63,17 +63,17 @@ public static void finishPipeline(IMessage message) { */ public static MessageGlobleTrace joinMessage(IMessage message) { MessageHeader messageHeader = message.getHeader(); - if (messageHeader.getMessageGloableTrace() == null) { - messageHeader.setMessageGloableTrace(new MessageGlobleTrace()); + if (messageHeader.getMessageGlobalTrace() == null) { + messageHeader.setMessageGlobalTrace(new MessageGlobleTrace()); } - return messageHeader.getMessageGloableTrace(); + return messageHeader.getMessageGlobalTrace(); } public static void clear(IMessage message) { MessageHeader messageHeader = message.getHeader(); - if (messageHeader.getMessageGloableTrace() == null) { - messageHeader.setMessageGloableTrace(new MessageGlobleTrace()); + if (messageHeader.getMessageGlobalTrace() == null) { + messageHeader.setMessageGlobalTrace(new MessageGlobleTrace()); } - messageHeader.getMessageGloableTrace().existFinishBranch = false; + messageHeader.getMessageGlobalTrace().existFinishBranch = false; } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/Re2Engine.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/Re2Engine.java index 75c6d272..f03b7793 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/Re2Engine.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/Re2Engine.java @@ -18,11 +18,9 @@ import com.google.re2j.Matcher; import com.google.re2j.Pattern; -import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Set; import org.apache.commons.lang3.StringUtils; @@ -36,47 +34,42 @@ */ public class Re2Engine implements IStreamRegex { - protected com.google.re2j.Pattern pattern; + protected Pattern pattern; - private Map> expressionMap = new HashMap<>(128); + protected Map nameMap = new HashMap<>(64); - private Map nameMap = new HashMap<>(128); + protected Map regexMap = new HashMap<>(64); protected Map unSupportMap = new HashMap<>(32); @Override public void addRegex(String regex, T context) { String groupName = "P" + nameMap.size(); - if (!expressionMap.containsKey(regex)) { - expressionMap.put(regex, new ArrayList<>()); - } - expressionMap.get(regex).add(context); - if (!nameMap.containsKey(regex)) { - nameMap.put(regex, groupName); - } + nameMap.put(groupName, context); + regexMap.put(groupName, regex); } @Override public void compile() { StringBuffer buffer = new StringBuffer(); - Iterator> iterator = nameMap.entrySet().iterator(); - com.google.re2j.Pattern testPattern; + Iterator> iterator = regexMap.entrySet().iterator(); + Pattern testPattern; while (iterator.hasNext()) { Map.Entry entry = iterator.next(); - String expression = entry.getKey(); - String groupName = entry.getValue(); + String groupName = entry.getKey(); + String regex = entry.getValue(); try { - testPattern = com.google.re2j.Pattern.compile(expression, com.google.re2j.Pattern.MULTILINE); + testPattern = Pattern.compile(regex, Pattern.MULTILINE); } catch (Exception e) { iterator.remove(); - unSupportMap.put(groupName, expression); + unSupportMap.put(groupName, regex); continue; } if (buffer.length() != 0) { buffer.append("|"); } - buffer.append("(?P<").append(groupName).append(">(").append(expression).append("))"); + buffer.append("(?P<").append(groupName).append(">(").append(regex).append("))"); } if (buffer.length() != 0) { - pattern = com.google.re2j.Pattern.compile(buffer.toString(), com.google.re2j.Pattern.MULTILINE & Pattern.CASE_INSENSITIVE); + pattern = Pattern.compile(buffer.toString(), Pattern.MULTILINE & Pattern.CASE_INSENSITIVE); } } @@ -99,9 +92,10 @@ private Set normalMatchSet(String content) { Iterator> iterator = unSupportMap.entrySet().iterator(); while (iterator.hasNext()) { Map.Entry entry = iterator.next(); + String groupName = entry.getKey(); String regex = entry.getValue(); if (StringUtil.matchRegexCaseInsensitive(content, regex)) { - matchedSet.addAll(expressionMap.get(regex)); + matchedSet.add(nameMap.get(groupName)); } } return matchedSet; @@ -125,15 +119,12 @@ private boolean normalMatchBoolean(String content) { } Set matchedSet = new HashSet<>(); Matcher matcher = pattern.matcher(content); - int index = 0; while (matcher.find()) { - Iterator> iterator = nameMap.entrySet().iterator(); + Iterator iterator = regexMap.keySet().iterator(); while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - String groupName = entry.getValue(); - String expression = entry.getKey(); + String groupName = iterator.next(); if (matcher.group(groupName) != null) { - matchedSet.addAll(expressionMap.get(expression)); + matchedSet.add(nameMap.get(groupName)); break; } } @@ -146,6 +137,6 @@ private boolean normalMatchBoolean(String content) { } @Override public int size() { - return expressionMap.values().stream().mapToInt(list -> list.size()).sum(); + return regexMap.size(); } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/TaskOptimization.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/TaskOptimization.java index 61b87dac..2f0bd955 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/TaskOptimization.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/TaskOptimization.java @@ -17,6 +17,7 @@ package org.apache.rocketmq.streams.common.optimization; import java.util.ArrayList; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.ServiceLoader; @@ -31,25 +32,23 @@ public class TaskOptimization { /** * Automatically parses pipelines, generates pre-filter fingerprints and expression estimates */ - protected transient IHomologousOptimization homologousOptimization; + protected transient volatile IHomologousOptimization homologousOptimization; public TaskOptimization(int homologousExpressionCaseSize, int preFingerprintCaseSize) { this.homologousExpressionCaseSize = homologousExpressionCaseSize; this.preFingerprintCaseSize = preFingerprintCaseSize; } - public void openOptimiazte(ChainPipeline... pipelines) { + public void openOptimization(ChainPipeline... pipelines) { if (pipelines == null) { return; } - List pipelineList = new ArrayList<>(); - for (ChainPipeline pipeline : pipelines) { - pipelineList.add(pipeline); - } - openOptimiazte(pipelineList); + List> pipelineList = new ArrayList<>(); + Collections.addAll(pipelineList, pipelines); + openOptimization(pipelineList); } - public void openOptimiazte(List pipelines) { + public void openOptimization(List> pipelines) { if (this.homologousOptimization == null) { synchronized (this) { if (this.homologousOptimization == null) { @@ -64,7 +63,7 @@ public void openOptimiazte(List pipelines) { } } - public void calculateOptimiazteExpression(IMessage message, AbstractContext context) { + public void calculateOptimizationExpression(IMessage message, AbstractContext context) { if (homologousOptimization != null) { homologousOptimization.calculate(message, context); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintCache.java index 8d5a072d..aae3c33c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintCache.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/FingerprintCache.java @@ -35,7 +35,7 @@ public class FingerprintCache { protected static FingerprintCache fingerprintCache; protected static int CACHE_SIZE = 5000000;//default cache size,support 3000000 log size - + public static String FIELD_VALUE_SPLIT_SIGN=";;;;;"; //key: namespace value:FingerprintMetric protected Map metricMap = new HashMap<>(); @@ -200,7 +200,7 @@ public static String creatFingerpringKey(IMessage message, String namespace, Str if (isFirst) { isFirst = false; } else { - sb.append(";"); + sb.append(FIELD_VALUE_SPLIT_SIGN); } sb.append(msgValue); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/PreFingerprint.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/PreFingerprint.java index 524ab03e..2c92249a 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/PreFingerprint.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/fingerprint/PreFingerprint.java @@ -24,7 +24,6 @@ import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; -import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; /** @@ -33,30 +32,41 @@ * execute filter in source stage, if match filte directly */ public class PreFingerprint { - protected transient String logFingerFieldNames;//如果有日志指纹,这里存储日志指纹的字段,启动时,通过属性文件加载 - protected transient String filterStageIdentification;//唯一标识一个filter + /** + * 如果有日志指纹,这里存储日志指纹的字段,启动时,通过属性文件加载 + */ + protected transient String logFingerFieldNames; + + /** + * 唯一标识一个filter + */ + protected transient String filterStageIdentification; - protected transient String sourceStageLable;//execute logfinger filter's stage, may be owner mutil branch stage or pipeline source - protected transient String nextStageLable;//the source stage's next stage lable + /** + * execute logfinger filter's stage, may be owner mutil branch stage or pipeline source + */ + protected transient String sourceStageLabel; + /** + * the source stage's next stage label + */ + protected transient String nextStageLabel; protected transient FingerprintCache fingerprintCache; protected transient int expressionCount = -1; - protected transient FilterChainStage filterChainStage; - protected transient List allPreviewFilterChainStage = new LinkedList<>(); + protected transient AbstractStage filterChainStage; + protected transient List> allPreviewFilterChainStage = new LinkedList<>(); - public PreFingerprint(String logFingerFieldNames, String filterStageIdentification, String sourceStageLable, - String nextStageLable, int expressionCount, AbstractStage filterChainStage, FingerprintCache fingerprintCache) { + public PreFingerprint(String logFingerFieldNames, String filterStageIdentification, String sourceStageLable, String nextStageLable, int expressionCount, AbstractStage filterChainStage, FingerprintCache fingerprintCache) { this.logFingerFieldNames = logFingerFieldNames; this.filterStageIdentification = filterStageIdentification; - this.sourceStageLable = sourceStageLable; - this.nextStageLable = nextStageLable; + this.sourceStageLabel = sourceStageLable; + this.nextStageLabel = nextStageLable; this.expressionCount = expressionCount; - this.filterChainStage = (FilterChainStage) filterChainStage; + this.filterChainStage = filterChainStage; this.fingerprintCache = fingerprintCache; } - public PreFingerprint(String logFingerFieldNames, String filterStageIdentification, String sourceStageLable, - String nextStageLable, AbstractStage filterChainStage, FingerprintCache fingerprintCache) { + public PreFingerprint(String logFingerFieldNames, String filterStageIdentification, String sourceStageLable, String nextStageLable, AbstractStage filterChainStage, FingerprintCache fingerprintCache) { this(logFingerFieldNames, filterStageIdentification, sourceStageLable, nextStageLable, -1, filterChainStage, fingerprintCache); } @@ -100,12 +110,12 @@ public String getLogFingerFieldNames() { return logFingerFieldNames; } - public String getSourceStageLable() { - return sourceStageLable; + public String getSourceStageLabel() { + return sourceStageLabel; } - public String getNextStageLable() { - return nextStageLable; + public String getNextStageLabel() { + return nextStageLabel; } public int getExpressionCount() { @@ -116,7 +126,7 @@ public void setExpressionCount(int expressionCount) { this.expressionCount = expressionCount; } - public FilterChainStage getFilterChainStage() { + public AbstractStage getFilterChainStage() { return filterChainStage; } @@ -126,15 +136,15 @@ public void setLogFingerFieldNames(Set logFingerFieldNames) { this.logFingerFieldNames = MapKeyUtil.createKey(",", fingers); } - public void addPreviwFilterChainStage(List filterChainStages) { + public void addPreviwFilterChainStage(List> filterChainStages) { this.allPreviewFilterChainStage.addAll(filterChainStages); } - public void addPreviwFilterChainStage(FilterChainStage filterChainStage) { + public void addPreviwFilterChainStage(AbstractStage filterChainStage) { this.allPreviewFilterChainStage.add(filterChainStage); } - public List getAllPreviewFilterChainStage() { + public List> getAllPreviewFilterChainStage() { return allPreviewFilterChainStage; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleTask.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleTask.java index 7bcb59a0..165578f7 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleTask.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/schedule/ScheduleTask.java @@ -76,11 +76,11 @@ public boolean canExecute() { initialDelaySecond = -1; return true; } - if (initialDelaySecond > 0 && (System.currentTimeMillis() - lastExecuteTime > initialDelaySecond)) { + if (initialDelaySecond > 0 && (System.currentTimeMillis() - lastExecuteTime > initialDelaySecond*1000)) { initialDelaySecond = -1; return true; } - if (System.currentTimeMillis() - lastExecuteTime > delaySecond) { + if (System.currentTimeMillis() - lastExecuteTime > delaySecond*1000) { return true; } isExecuting.compareAndSet(true, false); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ThreadPoolFactory.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ThreadPoolFactory.java index f1940264..5cbdc489 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ThreadPoolFactory.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/threadpool/ThreadPoolFactory.java @@ -18,14 +18,16 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; public class ThreadPoolFactory { public static ExecutorService createThreadPool(int coreSize){ ExecutorService executorService= new ThreadPoolExecutor(coreSize, coreSize, 1000*60L, TimeUnit.MILLISECONDS, - new SynchronousQueue(),new ThreadPoolExecutor.CallerRunsPolicy()); + new SynchronousQueue(), new DipperThreadFactory(), new ThreadPoolExecutor.CallerRunsPolicy()); return executorService; } @@ -33,7 +35,35 @@ public static ExecutorService createThreadPool(int coreSize){ public static ExecutorService createThreadPool(int min,int max){ ExecutorService executorService= new ThreadPoolExecutor(min, max, 1000*60L, TimeUnit.MILLISECONDS, - new SynchronousQueue(),new ThreadPoolExecutor.CallerRunsPolicy()); + new SynchronousQueue(),new DipperThreadFactory(), new ThreadPoolExecutor.CallerRunsPolicy()); return executorService; } + + + static class DipperThreadFactory implements ThreadFactory { + private static final AtomicInteger poolNumber = new AtomicInteger(1); + private final ThreadGroup group; + private final AtomicInteger threadNumber = new AtomicInteger(1); + private final String namePrefix; + + DipperThreadFactory() { + SecurityManager s = System.getSecurityManager(); + group = (s != null) ? s.getThreadGroup() : + Thread.currentThread().getThreadGroup(); + namePrefix = "dipper-pool-" + + poolNumber.getAndIncrement() + + "-thread-"; + } + + public Thread newThread(Runnable r) { + Thread t = new Thread(group, r, + namePrefix + threadNumber.getAndIncrement(), + 0); + if (t.isDaemon()) + t.setDaemon(false); + if (t.getPriority() != Thread.NORM_PRIORITY) + t.setPriority(Thread.NORM_PRIORITY); + return t; + } + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/AbstractMutilPipelineChainPipline.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/AbstractMutilPipelineChainPipline.java index 0787d420..3969ebf9 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/AbstractMutilPipelineChainPipline.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/AbstractMutilPipelineChainPipline.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Set; import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; import org.apache.rocketmq.streams.common.channel.source.systemmsg.NewSplitMessage; import org.apache.rocketmq.streams.common.channel.source.systemmsg.RemoveSplitMessage; @@ -31,28 +32,29 @@ import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.Context; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.topology.model.AbstractStage; import org.apache.rocketmq.streams.common.topology.model.IStageHandle; import org.apache.rocketmq.streams.common.topology.model.Pipeline; -import org.apache.rocketmq.streams.common.topology.stages.UnionChainStage; +import org.apache.rocketmq.streams.common.topology.stages.JoinChainStage; +import org.apache.rocketmq.streams.common.topology.stages.WindowChainStage; import org.apache.rocketmq.streams.common.utils.CollectionUtil; /** - * 如果某个节点有多个pipline构成,可以继承此类,如union,join节点 - * - * @param + * 新的解析已经废弃,主要兼容老的规则数据 */ +@Deprecated public abstract class AbstractMutilPipelineChainPipline extends ChainStage implements IAfterConfigurableRefreshListener { /** * pipeline name,这是一个汇聚节点,会有多个pipline,这里存的是pipline name */ protected List piplineNames = new ArrayList<>(); //每个pipline,对应一个消息来源,在消息头上会有消息来源的name,根据name转发数据 - protected Map piplineName2MsgSourceName; + protected Map> piplineName2MsgSourceName; /** * piplineNames的对象表示 */ - protected transient Map piplines = null; + protected transient Map> piplines = null; protected transient IStageHandle handle = new IStageHandle() { @Override @@ -63,31 +65,34 @@ protected IMessage doProcess(IMessage message, AbstractContext context) { String msgSourceName = message.getHeader().getMsgRouteFromLable(); if (piplines.size() > 0) { List messages = new ArrayList<>(); - Iterator> it = piplineName2MsgSourceName.entrySet().iterator(); + Iterator>> it = piplineName2MsgSourceName.entrySet().iterator(); while (it.hasNext()) { - Entry entry = it.next(); + Entry> entry = it.next(); String piplineName = entry.getKey(); - String value = entry.getValue(); - if (msgSourceName != null && msgSourceName.equals(value)) {//如果来源数据的标签和map中的相同,转发这条消息给对应的pipline - ChainPipeline pipline = piplines.get(piplineName); - IMessage copyMessage = message.deepCopy(); - //copyMessage.getMessageBody().put(ORI_MESSAGE_KEY,message.getMessageBody()); - // 保留一份最原始的数据,后续对字段的修改不影响这个字段 - Context newContext = new Context(copyMessage); - copyMessage.getHeader().setMsgRouteFromLable(msgSourceName); - boolean needReturn = executePipline(pipline, copyMessage, newContext, msgSourceName); - if (needReturn) { - return message; - } - if (newContext.isContinue()) { - if (newContext.isSplitModel()) { - messages.addAll(newContext.getSplitMessages()); - } else { - messages.add(copyMessage); + Set values = entry.getValue(); + for(String value:values){ + if (msgSourceName != null && msgSourceName.equals(value)) {//如果来源数据的标签和map中的相同,转发这条消息给对应的pipline + ChainPipeline pipline = piplines.get(piplineName); + IMessage copyMessage = message.deepCopy(); + //copyMessage.getMessageBody().put(ORI_MESSAGE_KEY,message.getMessageBody()); + // 保留一份最原始的数据,后续对字段的修改不影响这个字段 + Context newContext = new Context(copyMessage); + copyMessage.getHeader().setMsgRouteFromLable(msgSourceName); + boolean needReturn = executePipline(pipline, copyMessage, newContext, msgSourceName); + if (needReturn) { + return message; } + if (newContext.isContinue()) { + if (newContext.isSplitModel()) { + messages.addAll(newContext.getSplitMessages()); + } else { + messages.add(copyMessage); + } + } } } + } for (IMessage msg : messages) { msg.getHeader().setMsgRouteFromLable(msgSourceName); @@ -101,7 +106,7 @@ protected IMessage doProcess(IMessage message, AbstractContext context) { @Override public String getName() { - return UnionChainStage.class.getName(); + return AbstractMutilPipelineChainPipline.class.getName(); } }; @@ -168,12 +173,23 @@ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableS if (piplineNames == null) { return; } - Map piplineMap = new HashMap<>(); + Map> piplineMap = new HashMap<>(); for (String pipeLineName : piplineNames) { ChainPipeline chainPipline = configurableService.queryConfigurable(Pipeline.TYPE, pipeLineName); if (chainPipline != null) { piplineMap.put(chainPipline.getConfigureName(), chainPipline); } + List> stages= chainPipline.getStages(); + for(AbstractStage stage:stages){ + stage.setPipeline(getPipeline()); + if(WindowChainStage.class.isInstance(stage)){ + ((WindowChainStage)stage).getWindow().setFireReceiver(getReceiverAfterCurrentNode()); + }else if(JoinChainStage.class.isInstance(stage)){ + ((JoinChainStage) stage).doProcessAfterRefreshConfigurable(configurableService); + ((JoinChainStage)stage).getWindow().setFireReceiver(getReceiverAfterCurrentNode()); + } + } + } this.piplines = piplineMap; } @@ -192,15 +208,18 @@ public List getPiplines() { return piplines; } - public Map getPiplineName2MsgSourceName() { + public Map> getPiplineName2MsgSourceName() { return piplineName2MsgSourceName; } + public void setPiplineName2MsgSourceName( + Map> piplineName2MsgSourceName) { + this.piplineName2MsgSourceName = piplineName2MsgSourceName; + } + public ChainPipeline getPipeline(String pipelineName){ return this.piplines.get(pipelineName); } - public void setPiplineName2MsgSourceName(Map piplineName2MsgSourceName) { - this.piplineName2MsgSourceName = piplineName2MsgSourceName; - } + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainPipeline.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainPipeline.java index f60bf22e..96416044 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainPipeline.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainPipeline.java @@ -18,7 +18,9 @@ import com.alibaba.fastjson.JSONObject; import com.google.common.collect.Lists; +import java.io.File; import java.io.Serializable; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; @@ -37,22 +39,28 @@ import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.monitor.ConsoleMonitorManager; import org.apache.rocketmq.streams.common.monitor.IMonitor; import org.apache.rocketmq.streams.common.monitor.group.MonitorCommander; import org.apache.rocketmq.streams.common.optimization.IHomologousOptimization; import org.apache.rocketmq.streams.common.optimization.MessageGlobleTrace; import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; +import org.apache.rocketmq.streams.common.schedule.ScheduleManager; +import org.apache.rocketmq.streams.common.schedule.ScheduleTask; +import org.apache.rocketmq.streams.common.topology.metric.StageGroup; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; import org.apache.rocketmq.streams.common.topology.model.Pipeline; import org.apache.rocketmq.streams.common.utils.DipperThreadLocalUtil; +import org.apache.rocketmq.streams.common.utils.FileUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.PipelineHTMLUtil; import org.apache.rocketmq.streams.common.utils.PrintUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; /** * 数据流拓扑结构,包含了source 算子,sink */ -public class ChainPipeline extends Pipeline implements IAfterConfigurableRefreshListener, Serializable { +public class ChainPipeline extends Pipeline implements IAfterConfigurableRefreshListener, Serializable,Runnable { private static final long serialVersionUID = -5189371682717444347L; @@ -63,53 +71,48 @@ public class ChainPipeline extends Pipeline implements IA private transient int homologousExpressionCacheSize = 2000000; private transient int preFingerprintCacheSize = 2000000; - private transient IHomologousOptimization homologousOptimization; //对pipeline执行预编译的优化 - /** - * 是否自动启动channel + * 对pipeline执行预编译的优化 */ - protected boolean isAutoStart = false; + private transient IHomologousOptimization homologousOptimization; - /** - * pipeline状态,0,不启动,1-启动 - */ - protected Integer pipelineStatus; - - protected transient ISource source; + protected transient ISource source; /** * channel对应后续的stageName */ protected List channelNextStageLabel; - protected transient Map stageMap = new HashMap<>(); - - // protected transient AtomicBoolean initProcessor = new AtomicBoolean(false); + protected transient Map> stageMap = new HashMap<>(); private String channelName; - protected MetaData channelMetaData;//数据源输入格式,主要用于日志指纹过滤,如果没有则不做优化 + /** + * 数据源输入格式,主要用于日志指纹过滤,如果没有则不做优化 + */ + protected MetaData channelMetaData; /** * 是否发布,默认为true,关闭发布时,此字段为false,pipeline启动时应判断此字段是否为true,status默认都为1,status为0代表pipeline已被删除 */ - private boolean isPublish = false; protected transient AtomicBoolean hasStart = new AtomicBoolean(false); - public Integer getPipelineStatus() { - return pipelineStatus; - } + /** + * 为了图形化拓扑和监控使用 + */ + protected List stageGroups=new ArrayList<>(); + protected transient List rootStageGroups=new ArrayList<>(); + protected String createTableSQL; - public void setPipelineStatus(Integer pipelineStatus) { - this.pipelineStatus = pipelineStatus; - } /** * 启动一个channel,并给channel应用pipeline */ public void startChannel() { + //定时生成拓扑页面 + ScheduleManager.getInstance().regist(new ScheduleTask(0,10,this)); final String monitorName = createPipelineMonitorName(); if (isInitSuccess()) { if (!hasStart.compareAndSet(false, true)) { @@ -124,18 +127,18 @@ public void startChannel() { String isOpenOptimizationStr = ComponentCreator.getProperties().getProperty("homologous.optimization.switch"); boolean isOpenOptimization = true; if (StringUtil.isNotEmpty(isOpenOptimizationStr)) { - isOpenOptimization = Boolean.valueOf(isOpenOptimizationStr); + isOpenOptimization = Boolean.parseBoolean(isOpenOptimizationStr); } if (this.homologousOptimization == null && isOpenOptimization) { Iterable iterable = ServiceLoader.load(IHomologousOptimization.class); Iterator it = iterable.iterator(); String homologousExpressionCacheSizeStr = ComponentCreator.getProperties().getProperty("homologous.expression.cache.size"); if (StringUtil.isNotEmpty(homologousExpressionCacheSizeStr)) { - this.homologousExpressionCacheSize = Integer.valueOf(homologousExpressionCacheSizeStr); + this.homologousExpressionCacheSize = Integer.parseInt(homologousExpressionCacheSizeStr); } String preFingerprintCacheSizeStr = ComponentCreator.getProperties().getProperty("homologous.pre.fingerprint.cache.size"); if (StringUtil.isNotEmpty(preFingerprintCacheSizeStr)) { - this.preFingerprintCacheSize = Integer.valueOf(preFingerprintCacheSizeStr); + this.preFingerprintCacheSize = Integer.parseInt(preFingerprintCacheSizeStr); } if (it.hasNext()) { this.homologousOptimization = it.next(); @@ -145,13 +148,18 @@ public void startChannel() { try { AtomicLong COUNT = new AtomicLong(0); - Long startTime = System.currentTimeMillis(); - Boolean isPrintPipelineQPS = ComponentCreator.getPropertyBooleanValue("pipeline.qps.print"); + long startTime = System.currentTimeMillis(); + boolean isPrintPipelineQPS = ComponentCreator.getPropertyBooleanValue("pipeline.qps.print"); source.start((IStreamOperator) (message, context) -> { //每条消息一个,监控整个链路 IMonitor pipelineMonitorForStage = context.startMonitor(monitorName); pipelineMonitorForStage.setType(IMonitor.TYPE_DATAPROCESS); - message.getHeader().setPiplineName(this.getConfigureName()); + if (!message.getHeader().isSystemMessage()) { + //如果没有前置数据源则从消息里面取延迟 + //msg.put("__time__",message.getHeader().getEventMsgTime()); + ConsoleMonitorManager.getInstance().reportChannel(ChainPipeline.this, source, message); + } + message.getHeader().setPipelineName(this.getConfigureName()); //在正式执行逻辑之前, 基于同源的优化策略先进行计算 if (this.homologousOptimization != null) { this.homologousOptimization.calculate(message, context); @@ -184,7 +192,6 @@ public void startChannel() { } else { LOG.error("channel init failure, so can not start channel"); } - } private String createDuplicateKey(IMessage message) { @@ -202,13 +209,13 @@ private String createPipelineMonitorName() { /** * 可以替换某个阶段的阶段,而不用配置的阶段 * - * @param t - * @param context - * @param replaceStage - * @return + * @param t 数据 + * @param context 上下文 + * @param replaceStage stage + * @return stage执行后的结果 */ @Override - protected T doMessageInner(T t, AbstractContext context, AbstractStage... replaceStage) { + protected T doMessageInner(T t, AbstractContext context, AbstractStage... replaceStage) { if (this.duplicateCache != null && this.duplicateFields != null && !this.duplicateFields.isEmpty() && !t.getHeader().isSystemMessage()) { String duplicateKey = createDuplicateKey(t); Long cacheTime = this.duplicateCache.get(duplicateKey); @@ -236,33 +243,26 @@ protected T doMessageInner(T t, AbstractContext context, AbstractStage... replac } protected boolean isTopology(List nextStageLabel) { - if (nextStageLabel == null || nextStageLabel.size() == 0) { - return false; - } - return true; + return nextStageLabel != null && nextStageLabel.size() != 0; } public boolean isTopology() { return isTopology(this.channelNextStageLabel); } - public void doNextStages(AbstractContext context, String msgPrewSourceName, String currentLable, - List nextStageLabel, String prewSQLNodeName) { - + public void doNextStages(AbstractContext context, String msgPrevSourceName, String currentLabel, List nextStageLabel, String prevSQLNodeName) { if (!isTopology(nextStageLabel)) { return; } - - String oriMsgPrewSourceName = msgPrewSourceName; - List currentStageLables = nextStageLabel; - int size = currentStageLables.size(); - for (String lable : currentStageLables) { + String oriMsgPrewSourceName = msgPrevSourceName; + int size = nextStageLabel.size(); + for (String lable : nextStageLabel) { AbstractContext copyContext = context; if (size > 1) { copyContext = context.copy(); } T msg = (T) copyContext.getMessage(); - AbstractStage oriStage = stageMap.get(lable); + AbstractStage oriStage = stageMap.get(lable); if (oriStage == null) { if (stages != null && stages.size() > 0) { synchronized (this) { @@ -279,50 +279,65 @@ public void doNextStages(AbstractContext context, String msgPrewSourceName, Stri } } AbstractStage stage = oriStage; - if (filterByPreFingerprint(msg, copyContext, currentLable, lable)) { + if (filterByPreFingerprint(msg, copyContext, currentLabel, lable)) { continue; } + if (!msg.getHeader().isSystemMessage()) { + ConsoleMonitorManager.getInstance().reportInput(stage, msg); + } + //boolean needFlush = needFlush(msg); if (StringUtil.isNotEmpty(oriMsgPrewSourceName)) { msg.getHeader().setMsgRouteFromLable(oriMsgPrewSourceName); } - /** - * 主要用于调试,这里进入一个新的sqlnode 了 - */ - //if (isNewSQLNode(stage, prewSQLNodeName) & msg.getHeader().isSystemMessage() == false) { - // if (LOG.isDebugEnabled()) { - // LOG.debug(msg.getHeader().getTraceId() + " " + prewSQLNodeName + "->" + stage.getOwnerSqlNodeTableName()); - // - // } - //} boolean isContinue = executeStage(stage, msg, copyContext); if (!isContinue) { - /** - * 只要执行到了window分支都不应该被过滤 - */ + //只要执行到了window分支都不应该被过滤 if (stage.isAsyncNode() && !msg.getHeader().isSystemMessage()) { MessageGlobleTrace.finishPipeline(msg); } - continue; } else { + if (!msg.getHeader().isSystemMessage()) { + ConsoleMonitorManager.getInstance().reportOutput(stage, msg, ConsoleMonitorManager.MSG_FLOWED, null); + } if (stage instanceof ChainStage) { - ChainStage chainStage = (ChainStage) stage; + ChainStage chainStage = (ChainStage) stage; String msgSourceName = chainStage.getMsgSourceName(); if (StringUtil.isNotEmpty(msgSourceName)) { - msgPrewSourceName = msgSourceName; + msgPrevSourceName = msgSourceName; } - } - List labels = stage.doRoute(msg); - if (labels == null || labels.size() == 0) { - if (!msg.getHeader().isSystemMessage()) { - MessageGlobleTrace.finishPipeline(msg); + + if (copyContext.isSplitModel()) { + List messageList = copyContext.getSplitMessages(); + int splitMessageOffset = 0; + for (IMessage message : messageList) { + AbstractContext abstractContext = copyContext.copy(); + abstractContext.closeSplitMode(message); + message.getHeader().setMsgRouteFromLable(msg.getHeader().getMsgRouteFromLable()); + message.getHeader().addLayerOffset(splitMessageOffset); + splitMessageOffset++; + List labels = stage.doRoute(message); + if (labels == null || labels.size() == 0) { + if (!message.getHeader().isSystemMessage()) { + MessageGlobleTrace.finishPipeline(message); + } + continue; + } + doNextStages(abstractContext, msgPrevSourceName, stage.getLabel(), labels, stage.getOwnerSqlNodeTableName()); } - continue; + } else { + List labels = stage.doRoute(msg); + if (labels == null || labels.size() == 0) { + if (!msg.getHeader().isSystemMessage()) { + MessageGlobleTrace.finishPipeline(msg); + } + continue; + } + doNextStages(copyContext, msgPrevSourceName, stage.getLabel(), labels, stage.getOwnerSqlNodeTableName()); } - doNextStages(copyContext, msgPrewSourceName, stage.getLabel(), labels, stage.getOwnerSqlNodeTableName()); } } } @@ -359,7 +374,7 @@ protected boolean executeStage(final AbstractStage stage, T t, AbstractContext c } catch (Exception e) { e.printStackTrace(); //优化日志量 - // LOG.error("execute stage error " + stage.getConfigureName(), e); + //LOG.error("execute stage error " + stage.getConfigureName(), e); stageMonitor.occureError(e, "execute stage error " + stage.getConfigureName(), e.getMessage()); stageMonitor.setSampleData(context).put("stage_info", createStageInfo(stage)); return false; @@ -419,9 +434,17 @@ public void setSource(ISource source) { @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { createStageMap(); - ISource source = configurableService.queryConfigurable(ISource.TYPE, channelName); + Map stageGroupMap=createStageGroupMap(); + for(StageGroup stageGroup:this.stageGroups){ + stageGroup.init(this.stageMap,stageGroupMap); + if(stageGroup.getParent()==null&&!this.rootStageGroups.contains(stageGroup)){ + this.rootStageGroups.add(stageGroup); + } + } + + ISource source = configurableService.queryConfigurable(ISource.TYPE, channelName); this.source = source; - for (AbstractStage stage : getStages()) { + for (AbstractStage stage : getStages()) { stage.setPipeline(this); if (stage instanceof IAfterConfigurableRefreshListener) { if (!stage.isInitSuccess() && !this.isInitSuccess()) { @@ -429,18 +452,10 @@ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableS return; } IAfterConfigurableRefreshListener afterConfigurableRefreshListener = (IAfterConfigurableRefreshListener) stage; - afterConfigurableRefreshListener.doProcessAfterRefreshConfigurable(configurableService); - } } - if (source != this.source && this.source != null) { - this.hasStart.set(false); - this.source = source; - startChannel(); - } - if (source instanceof AbstractConfigurable) { AbstractConfigurable abstractConfigurable = (AbstractConfigurable) source; if (!abstractConfigurable.isInitSuccess() && this.isInitSuccess()) { @@ -449,11 +464,6 @@ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableS } } - //修改发布状态为true或设置自动启动,需要调用startChannel - if ((isAutoStart || isPublish()) && isInitSuccess()) { - startChannel(); - } - //增加去重的逻辑 String duplicateFieldNameStr = ComponentCreator.getProperties().getProperty(getConfigureName() + ".duplicate.fields.names"); if (duplicateFieldNameStr != null && !duplicateFieldNameStr.isEmpty()) { @@ -472,22 +482,22 @@ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableS } - public Map createStageMap() { - for (AbstractStage stage : getStages()) { + private Map createStageGroupMap() { + Map map=new HashMap<>(); + for(StageGroup stageGroup:stageGroups){ + map.put(stageGroup.getConfigureName(),stageGroup); + } + return map; + } + + public Map> createStageMap() { + for (AbstractStage stage : getStages()) { stageMap.put(stage.getLabel(), stage); stage.setPipeline(this); } return stageMap; } - public boolean isAutoStart() { - return isAutoStart; - } - - public void setAutoStart(boolean autoStart) { - isAutoStart = autoStart; - } - public List getChannelNextStageLabel() { return channelNextStageLabel; } @@ -505,7 +515,7 @@ public String toString() { sb.append(source.toString()).append(LINE); } if (stages != null) { - for (AbstractStage stage : stages) { + for (AbstractStage stage : stages) { sb.append(stage.toString()); } @@ -515,13 +525,17 @@ public String toString() { @Override public void destroy() { - if (source != null) { + if (source != null && hasStart.compareAndSet(true, false)) { source.destroy(); } super.destroy(); } - - public Map getStageMap() { + public void addStageGroup(StageGroup stageGroup){ + if(this.stageGroups!=null){ + this.stageGroups.add(stageGroup); + } + } + public Map> getStageMap() { return stageMap; } @@ -529,14 +543,6 @@ public Boolean getHasStart() { return hasStart.get(); } - public boolean isPublish() { - return isPublish; - } - - public void setPublish(boolean publish) { - isPublish = publish; - } - public MetaData getChannelMetaData() { return channelMetaData; } @@ -552,4 +558,37 @@ public String getChannelName() { public void setChannelName(String channelName) { this.channelName = channelName; } + + public List getStageGroups() { + return stageGroups; + } + + public void setStageGroups(List stageGroups) { + this.stageGroups = stageGroups; + } + + public List getRootStageGroups() { + return rootStageGroups; + } + + public String getCreateTableSQL() { + return createTableSQL; + } + + public void setCreateTableSQL(String createTableSQL) { + this.createTableSQL = createTableSQL; + } + + @Override public void run() { + String filePath=FileUtil.getJarPath(); + if(StringUtil.isEmpty(filePath)){ + filePath="/tmp"; + } + filePath=filePath+ File.separator+getConfigureName()+".html"; + String html=PipelineHTMLUtil.createHTML(this); + synchronized (this){ + FileUtil.write(filePath,html); + LOG.info(getConfigureName()+" create pipeline html success in "+filePath); + } + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainStage.java index 3faf61ce..ac867bf7 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/ChainStage.java @@ -36,10 +36,14 @@ public abstract class ChainStage extends AbstractStage { @Changeable protected String entityName; + + //用于展示,辅助说明stage 在sql中的作用,如where函数解析,select 函数解析,select 字段排除等 + protected String discription; + /** * 是否取消IAfterConfigurableRefreshListener的执行,如成员变量通过set设置时,可以使用 */ - protected boolean cancelAfterConfigurableRefreshListerner = false; + protected boolean cancelAfterConfigurableRefreshListener = false; public String getEntityName() { return entityName; @@ -49,12 +53,12 @@ public void setEntityName(String entityName) { this.entityName = entityName; } - public boolean isCancelAfterConfigurableRefreshListerner() { - return cancelAfterConfigurableRefreshListerner; + public boolean isCancelAfterConfigurableRefreshListener() { + return cancelAfterConfigurableRefreshListener; } - public void setCancelAfterConfigurableRefreshListerner(boolean cancelAfterConfigurableRefreshListerner) { - this.cancelAfterConfigurableRefreshListerner = cancelAfterConfigurableRefreshListerner; + public void setCancelAfterConfigurableRefreshListener(boolean cancelAfterConfigurableRefreshListener) { + this.cancelAfterConfigurableRefreshListener = cancelAfterConfigurableRefreshListener; } /** @@ -64,8 +68,8 @@ public void setCancelAfterConfigurableRefreshListerner(boolean cancelAfterConfig * @param context * @param pipelines */ - public void sendSystem(IMessage message, AbstractContext context, Collection pipelines) { - if (message.getHeader().isSystemMessage() == false) { + public void sendSystem(IMessage message, AbstractContext context, Collection> pipelines) { + if (!message.getHeader().isSystemMessage()) { return; } if (pipelines != null && pipelines.size() > 0) { @@ -83,10 +87,10 @@ public void sendSystem(IMessage message, AbstractContext context, Collection set = new HashSet<>(); + Set> set = new HashSet<>(); for (Pipeline pipeline : pipelines) { if (pipeline != null) { set.add((ChainPipeline) pipeline); @@ -96,8 +100,14 @@ public void sendSystem(IMessage message, AbstractContext context, Pipeline... pi } protected SectionPipeline getReceiverAfterCurrentNode() { - SectionPipeline receiver = new SectionPipeline((ChainPipeline) getPipeline(), this); - return receiver; + return new SectionPipeline((ChainPipeline) getPipeline(), this); } + public String getDiscription() { + return discription; + } + + public void setDiscription(String discription) { + this.discription = discription; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/builder/PipelineBuilder.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/builder/PipelineBuilder.java index 87f8cb79..b1e85672 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/builder/PipelineBuilder.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/builder/PipelineBuilder.java @@ -22,17 +22,22 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.rocketmq.streams.common.channel.impl.view.ViewSink; import org.apache.rocketmq.streams.common.channel.sink.ISink; import org.apache.rocketmq.streams.common.channel.source.ISource; import org.apache.rocketmq.streams.common.configurable.AbstractConfigurable; import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.common.configurable.IConfigurableService; import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.model.NameCreatorContext; import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.ChainStage; +import org.apache.rocketmq.streams.common.topology.metric.StageGroup; +import org.apache.rocketmq.streams.common.topology.model.AbstractStage; import org.apache.rocketmq.streams.common.topology.model.Pipeline; import org.apache.rocketmq.streams.common.topology.stages.OutputChainStage; -import org.apache.rocketmq.streams.common.utils.NameCreatorUtil; +import org.apache.rocketmq.streams.common.topology.stages.ViewChainStage; import org.apache.rocketmq.streams.common.utils.StringUtil; public class PipelineBuilder implements Serializable { @@ -75,10 +80,20 @@ public class PipelineBuilder implements Serializable { protected String parentTableName; /** - * 设置这个值后,后面的所有逻辑都不再继续 + * 主要用在双流join的右流,是否是右流 */ - protected boolean isBreak = false; + protected boolean isRightJoin = false; + + + protected String rootTableName;//SQL Tree保存的是root tablename + + /** + * 主要用于把sql 按create view聚合,然后分层展开,便于监控和排错,主要用于sql解析时,完成stage的分组和层次关系 + */ + protected StageGroup currentStageGroup; + + protected StageGroup parentStageGroup; public PipelineBuilder(String namespace, String pipelineName) { pipeline.setNameSpace(namespace); @@ -165,12 +180,16 @@ public ChainStage createStage(ISink sink) { * @return */ public OutputChainStage addOutput(ISink sink) { - if (isBreak) { - return null; + OutputChainStage outputChainStage = new OutputChainStage<>();; + if(ViewSink.class.isInstance(sink)){ + + outputChainStage=new ViewChainStage(); } - OutputChainStage outputChainStage = new OutputChainStage<>(); sink.addConfigurables(this); outputChainStage.setSink(sink); + if (StringUtil.isEmpty(outputChainStage.getLabel())) { + outputChainStage.setLabel(createConfigurableName(outputChainStage.getType())); + } pipeline.addChainStage(outputChainStage); return outputChainStage; } @@ -181,9 +200,6 @@ public OutputChainStage addOutput(ISink sink) { * @param configurable */ public void addNameList(IConfigurable configurable) { - if (isBreak) { - return; - } addConfigurables(configurable); } @@ -193,13 +209,7 @@ public void addNameList(IConfigurable configurable) { * @param stageChainBuilder */ public ChainStage addChainStage(IStageBuilder stageChainBuilder) { - if (isBreak) { - return null; - } - ChainStage chainStage = stageChainBuilder.createStageChain(this); - stageChainBuilder.addConfigurables(this);// 这句一定要在addChainStage前,会默认赋值namespace和name - pipeline.addChainStage(chainStage); - return chainStage; + return createStage(stageChainBuilder); } /** @@ -209,16 +219,13 @@ public ChainStage addChainStage(IStageBuilder stageChainBuilder) * @return */ public String createConfigurableName(String type) { - return NameCreatorUtil.createNewName(this.pipelineName, type); + return NameCreatorContext.get().createNewName(this.pipelineName, type); } /** * 保存中间产生的结果 */ public void addConfigurables(IConfigurable configurable) { - if (isBreak) { - return; - } if (configurable != null) { if (StringUtil.isEmpty(configurable.getNameSpace())) { configurable.setNameSpace(getPipelineNameSpace()); @@ -237,9 +244,7 @@ public void addConfigurables(IConfigurable configurable) { } public void addConfigurables(Collection configurables) { - if (isBreak) { - return; - } + if (configurables != null) { for (IConfigurable configurable : configurables) { addConfigurables(configurable); @@ -253,21 +258,18 @@ public void addConfigurables(Collection configurables) * @param nextStages */ public void setTopologyStages(ChainStage currentChainStage, List nextStages) { - if (isBreak) { - return; - } if (nextStages == null) { return; } - List lableNames = new ArrayList<>(); + List labelNames = new ArrayList<>(); for (ChainStage stage : nextStages) { - lableNames.add(stage.getLabel()); + labelNames.add(stage.getLabel()); } if (currentChainStage == null) { - this.pipeline.setChannelNextStageLabel(lableNames); + this.pipeline.setChannelNextStageLabel(labelNames); } else { - currentChainStage.setNextStageLabels(lableNames); + currentChainStage.getNextStageLabels().addAll(labelNames); for (ChainStage stage : nextStages) { stage.getPrevStageLabels().add(currentChainStage.getLabel()); } @@ -280,9 +282,6 @@ public void setTopologyStages(ChainStage currentChainStage, List * @param nextStage */ public void setTopologyStages(ChainStage currentChainStage, ChainStage nextStage) { - if (isBreak) { - return; - } List stages = new ArrayList<>(); stages.add(nextStage); setTopologyStages(currentChainStage, stages); @@ -301,10 +300,10 @@ public ChainPipeline getPipeline() { } public void setHorizontalStages(ChainStage stage) { - if (isBreak) { - return; - } - List stages = new ArrayList<>(); +// if (isBreak) { +// return; +// } + List> stages = new ArrayList<>(); stages.add(stage); setHorizontalStages(stages); } @@ -314,16 +313,16 @@ public void setHorizontalStages(ChainStage stage) { * * @param stages */ - public void setHorizontalStages(List stages) { - if (isBreak) { - return; - } + public void setHorizontalStages(List> stages) { if (stages == null) { return; } List lableNames = new ArrayList<>(); Map> lableName2Stage = new HashMap(); for (ChainStage stage : stages) { + if(stage==null){ + continue; + } lableNames.add(stage.getLabel()); lableName2Stage.put(stage.getLabel(), stage); } @@ -332,11 +331,21 @@ public void setHorizontalStages(List stages) { this.pipeline.setChannelNextStageLabel(lableNames); for (String lableName : lableNames) { ChainStage chainStage = lableName2Stage.get(lableName); - chainStage.getPrevStageLabels().add(this.pipeline.getChannelName()); + if(this.pipeline.getChannelName()!=null){ + chainStage.getPrevStageLabels().add(this.pipeline.getChannelName()); + } } } else { - currentChainStage.setNextStageLabels(lableNames); + if(currentChainStage.getNextStageLabels()==null){ + currentChainStage.setNextStageLabels(new ArrayList<>()); + } for (String lableName : lableNames) { + if(StringUtil.isEmpty(lableName)){ + continue; + } + if(!currentChainStage.getNextStageLabels().contains(lableName)){ + currentChainStage.getNextStageLabels().add(lableName); + } ChainStage chainStage = lableName2Stage.get(lableName); List prewLables = chainStage.getPrevStageLabels(); if (!prewLables.contains(this.currentChainStage.getLabel())) { @@ -347,6 +356,18 @@ public void setHorizontalStages(List stages) { } } + public List> getFirstStages(){ + Map> stageMap= pipeline.createStageMap(); + List> stages=new ArrayList<>(); + List firstLables=pipeline.getChannelNextStageLabel(); + if(firstLables==null){ + return null; + } + for(String lableName:firstLables){ + stages.add((ChainStage) stageMap.get(lableName)); + } + return stages; + } public void setCurrentChainStage(ChainStage currentChainStage) { this.currentChainStage = currentChainStage; } @@ -375,11 +396,35 @@ public void setParentTableName(String parentTableName) { this.parentTableName = parentTableName; } - public boolean isBreak() { - return isBreak; + public boolean isRightJoin() { + return isRightJoin; + } + + public void setRightJoin(boolean rightJoin) { + isRightJoin = rightJoin; + } + + public String getRootTableName() { + return rootTableName; + } + + public void setRootTableName(String rootTableName) { + this.rootTableName = rootTableName; + } + + public StageGroup getCurrentStageGroup() { + return currentStageGroup; + } + + public void setCurrentStageGroup(StageGroup currentStageGroup) { + this.currentStageGroup = currentStageGroup; + } + + public StageGroup getParentStageGroup() { + return parentStageGroup; } - public void setBreak(boolean aBreak) { - isBreak = aBreak; + public void setParentStageGroup(StageGroup parentStageGroup) { + this.parentStageGroup = parentStageGroup; } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/NotFireReason.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/NotFireReason.java new file mode 100644 index 00000000..cc7db515 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/NotFireReason.java @@ -0,0 +1,176 @@ +/* + * 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.rocketmq.streams.common.topology.metric; + +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.optimization.IHomologousOptimization; +import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.utils.JsonableUtil; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; + +public class NotFireReason { + protected List oriFilterFieldNames=new ArrayList<>();//参与过滤的字段名 + protected Map oriFilterFields=new HashMap<>();//原始 + + + protected List expressions=new ArrayList<>();//过滤失败的表达式 + protected Map filterFields=new HashMap<>();//参与过滤的字段名和值 + protected Map> filterFieldName2ETLScriptList=new HashMap<>();//过滤字段和ETL脚本 + private List filterFieldNames=new ArrayList<>();//过滤字段列表 + + protected transient ChainPipeline pipeline; + protected transient FilterChainStage stage; + public NotFireReason(FilterChainStage stage,String fieldValues){ + this.stage=stage; + this.pipeline=(ChainPipeline) stage.getPipeline(); + String logFingerFieldNames=this.stage.getPreFingerprint().getLogFingerFieldNames(); + String[] values=logFingerFieldNames.split(","); + for(String oriFieldName:values){ + oriFilterFieldNames.add(oriFieldName); + } + values=fieldValues.split(FingerprintCache.FIELD_VALUE_SPLIT_SIGN); + for(int i=0;i> filterFieldName2ETLScriptList,Map filterFieldName2OriFieldName,List expressions,List filterFieldNames){ + this.expressions.addAll(expressions); + Map> etlScript=new HashMap<>(); + this.filterFieldNames.addAll(filterFieldNames); + Map oriFilterFields=new HashMap<>(); + List oriFilterFieldNames=new ArrayList<>(); + for(String filteFieldName:filterFieldNames){ + String oriFieldName=filterFieldName2OriFieldName.get(filteFieldName); + String oriFieldValue=this.oriFilterFields.get(oriFieldName); + if(oriFieldName!=null&&oriFieldValue!=null){ + oriFilterFields.put(oriFieldName,oriFieldValue); + } + String filterValue=message.getMessageBody().getString(filteFieldName); + if(filterValue!=null){ + filterFields.put(filteFieldName,filterValue); + } + + if(oriFieldName!=null){ + oriFilterFieldNames.add(oriFieldName); + } + List etl=filterFieldName2ETLScriptList.get(filteFieldName); + if(etl!=null){ + etlScript.put(oriFieldName,etl); + } + + } + this.filterFieldName2ETLScriptList=etlScript; + this.oriFilterFields=oriFilterFields; + this.oriFilterFieldNames=oriFilterFieldNames; + } + + public List getOriFilterFieldNames() { + return oriFilterFieldNames; + } + + public void setOriFilterFieldNames(List oriFilterFieldNames) { + this.oriFilterFieldNames = oriFilterFieldNames; + } + + public Map getOriFilterFields() { + return oriFilterFields; + } + + public void setOriFilterFields(Map oriFilterFields) { + this.oriFilterFields = oriFilterFields; + } + + public List getExpressions() { + return expressions; + } + + public void setExpressions(List expressions) { + this.expressions = expressions; + } + + public Map getFilterFields() { + return filterFields; + } + + public void setFilterFields(Map filterFields) { + this.filterFields = filterFields; + } + + public Map> getFilterFieldName2ETLScriptList() { + return filterFieldName2ETLScriptList; + } + + public void setFilterFieldName2ETLScriptList( + Map> filterFieldName2ETLScriptList) { + this.filterFieldName2ETLScriptList = filterFieldName2ETLScriptList; + } + + public List getFilterFieldNames() { + return filterFieldNames; + } + + public void setFilterFieldNames(List filterFieldNames) { + this.filterFieldNames = filterFieldNames; + } + + public FilterChainStage getStage() { + return stage; + } + + public ChainPipeline getPipeline() { + return pipeline; + } + @Override public String toString() { + return JsonableUtil.formatJson(toJson()); + } + public JSONObject toJson() { + JSONObject jsonObject=new JSONObject(); + jsonObject.put("expression", MapKeyUtil.createKey("\n",this.expressions)); + + if(CollectionUtil.isNotEmpty(this.filterFields)){ + JSONObject filterFields=new JSONObject(); + filterFields.putAll(this.filterFields); + jsonObject.put("field current value",filterFields); + } + + + if(CollectionUtil.isNotEmpty(this.oriFilterFields)){ + JSONObject oriFilterFields=new JSONObject(); + oriFilterFields.putAll(this.oriFilterFields); + jsonObject.put("field original value",oriFilterFields); + + } + + if(CollectionUtil.isNotEmpty(this.filterFieldName2ETLScriptList)){ + JSONObject etl=new JSONObject(); + etl.putAll(this.filterFieldName2ETLScriptList); + jsonObject.put("field etl",etl); + } + + return jsonObject; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageGroup.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageGroup.java new file mode 100644 index 00000000..d1a1fbb4 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageGroup.java @@ -0,0 +1,248 @@ +/* + * 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.rocketmq.streams.common.topology.metric; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; + +import org.apache.rocketmq.streams.common.model.NameCreatorContext; +import org.apache.rocketmq.streams.common.topology.model.AbstractStage; + +public class StageGroup extends BasedConfigurable { + public static String TYPE="StageGroup"; + + /** + * 展示在拓扑中 + */ + protected String viewName; + /** + * stage info + */ + protected String startLable; + protected String endLable; + protected List allStageLables; + + protected transient AbstractStage startStage; + protected transient AbstractStage endStage; + protected transient List> allStages; + + + protected List childrenNames=new ArrayList<>(); + protected String parentName; + + protected transient List children=new ArrayList<>(); + protected transient StageGroup parent; + + /** + * SQL info + */ + + protected String sql; + protected String subTableName; + + /** + * metric info + */ + + protected double qps;//1分钟qps + protected long inCount; + protected long outCount; + protected long avgCost; + protected long maxCost; + + public StageGroup(){ + setType(TYPE); + } + + public StageGroup(AbstractStage startStage, AbstractStage endStage, List> allStages){ + setType(TYPE); + this.setConfigureName(NameCreatorContext.get().createName("StageGroup")); +// this.setNameSpace(namespace); + setStartStage(startStage); + setEndStage(endStage); + if(allStages!=null){ + List allStageLables=new ArrayList<>(); + for(AbstractStage stage:allStages){ + allStageLables.add(stage.getLabel()); + } + this.allStageLables=allStageLables; + this.allStages=allStages; + } + } + + /** + * 在序列化时做反序列化 + * @param stageMap + */ + public void init(Map> stageMap,Map stageGroupMap){ + this.startStage=stageMap.get(this.startLable); + this.endStage=stageMap.get(this.endLable); + this.parent=stageGroupMap.get(this.parentName); + List children=new ArrayList<>(); + for(String name:this.childrenNames){ + children.add(stageGroupMap.get(name)); + } + this.children=children; + + List> allStages=new ArrayList<>(); + List tmpStageLables=new ArrayList<>(); + tmpStageLables.addAll(this.allStageLables); + Iterator it = tmpStageLables.iterator(); + while (it.hasNext()){ + String name=it.next(); + AbstractStage stage=stageMap.get(name); + allStages.add(stage); + stage.setStageGroup(this); + } + this.allStages=allStages; + } + + public void calculateMetric(){ + inCount=startStage.getStageMetric().getInCount(); + qps=startStage.getStageMetric().getQps(); + outCount=endStage.getStageMetric().getOutCount(); + } + + public String getStartLable() { + return startLable; + } + + public void setStartLable(String startLable) { + this.startLable = startLable; + } + + public String getEndLable() { + return endLable; + } + + public void setEndLable(String endLable) { + this.endLable = endLable; + } + + + public String getSql() { + return sql; + } + + public void setSql(String sql) { + this.sql = sql; + } + + public String getSubTableName() { + return subTableName; + } + + public void setSubTableName(String subTableName) { + this.subTableName = subTableName; + } + + public double getQps() { + return qps; + } + + public long getInCount() { + return inCount; + } + + public long getOutCount() { + return outCount; + } + + public long getAvgCost() { + return avgCost; + } + + public long getMaxCost() { + return maxCost; + } + + public void setStartStage(AbstractStage startStage) { + this.startStage = startStage; + this.startLable=startStage.getLabel(); + } + + public void setEndStage(AbstractStage endStage) { + this.endStage = endStage; + this.endLable=endStage.getLabel(); + } + + public void setChildren(List children) { + this.children = children; + } + + public StageGroup getParent() { + return parent; + } + + public void setParent(StageGroup parent) { + parent.children.add(this); + this.parent = parent; + this.parentName=this.parent.getConfigureName(); + parent.childrenNames.add(this.getConfigureName()); + } + + public List getChildrenNames() { + return childrenNames; + } + + public void setChildrenNames(List childrenNames) { + this.childrenNames = childrenNames; + } + + public String getParentName() { + return parentName; + } + + public void setParentName(String parentName) { + this.parentName = parentName; + } + + public AbstractStage getStartStage() { + return startStage; + } + + public AbstractStage getEndStage() { + return endStage; + } + + public List getChildren() { + return children; + } + + public List getAllStageLables() { + return allStageLables; + } + + public void setAllStageLables(List allStageLables) { + this.allStageLables = allStageLables; + } + + public String getViewName() { + return viewName; + } + + public void setViewName(String viewName) { + this.viewName = viewName; + } + + public List> getAllStages() { + return allStages; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageMetric.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageMetric.java new file mode 100644 index 00000000..046f826e --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/metric/StageMetric.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.rocketmq.streams.common.topology.metric; + +import com.alibaba.fastjson.JSONArray; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.utils.JsonableUtil; + +public class StageMetric { + /** + * metric info + */ + protected AtomicLong inCount=new AtomicLong(0); + protected AtomicLong outCount=new AtomicLong(0); + protected transient Long firstReceiveTime; + protected double qps; + + protected long maxCostTime; + protected long avgCostTime; + protected transient long sumCostTime; + + protected List notFireReasons=new ArrayList<>(); + + public long startCalculate(IMessage msg){ + if(firstReceiveTime==null){ + firstReceiveTime=System.currentTimeMillis(); + } + long countValue=inCount.incrementAndGet(); + long timeGap=(System.currentTimeMillis()-firstReceiveTime)/1000; + if(timeGap<1) { + return System.currentTimeMillis(); + } + qps=countValue/timeGap; + return System.currentTimeMillis(); + } + + public void endCalculate(long startTime){ + long cost=System.currentTimeMillis()-startTime; + this.sumCostTime+=cost; + this.avgCostTime=this.sumCostTime/inCount.get(); + if(this.maxCostTime100){ + notFireReasons.remove(0); + } + + + } + + public Long getFirstReceiveTime() { + return firstReceiveTime; + } + + public void setFirstReceiveTime(Long firstReceiveTime) { + this.firstReceiveTime = firstReceiveTime; + } + + public double getQps() { + return qps; + } + + public void setQps(double qps) { + this.qps = qps; + } + + public long getMaxCostTime() { + return maxCostTime; + } + + public void setMaxCostTime(long maxCostTime) { + this.maxCostTime = maxCostTime; + } + + public long getAvgCostTime() { + return avgCostTime; + } + + public void setAvgCostTime(long avgCostTime) { + this.avgCostTime = avgCostTime; + } + + public long getSumCostTime() { + return sumCostTime; + } + + public void setSumCostTime(long sumCostTime) { + this.sumCostTime = sumCostTime; + } + + public Long getInCount() { + return inCount.get(); + } + public Long getOutCount() { + return outCount.get(); + } + + public List getNotFireReasons() { + return notFireReasons; + } + + public String createNotFireReason() { + JSONArray jsonArray=new JSONArray(); + for(NotFireReason notFireReason:this.notFireReasons){ + jsonArray.add(notFireReason.toJson()); + } + return JsonableUtil.formatJson(jsonArray); + } + + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractStage.java index dced52e4..2434b838 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/AbstractStage.java @@ -20,7 +20,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; +import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; @@ -33,12 +33,13 @@ import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.metric.StageGroup; +import org.apache.rocketmq.streams.common.topology.metric.StageMetric; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.common.utils.TraceUtil; -public abstract class AbstractStage extends BasedConfigurable - implements IStreamOperator, ISystemMessageProcessor { +public abstract class AbstractStage extends BasedConfigurable implements IStreamOperator, ISystemMessageProcessor { protected String filterFieldNames; private static final Log LOG = LogFactory.getLog(AbstractStage.class); @@ -47,12 +48,14 @@ public abstract class AbstractStage extends BasedConfigurabl protected transient String name; - //是否关闭拆分模式,多条日志会合并在一起,字段名为splitDataFieldName + /** + * 是否关闭拆分模式,多条日志会合并在一起,字段名为splitDataFieldName + */ protected boolean closeSplitMode = false; protected String splitDataFieldName; - protected transient Pipeline pipeline; + protected transient Pipeline pipeline; /** * 设置路由label,当需要做路由选择时需要设置 @@ -79,30 +82,44 @@ public abstract class AbstractStage extends BasedConfigurabl */ protected String ownerSqlNodeTableName; + /** + * 主要用于排错,把stage按sql分组,可以快速定位问题 + */ + protected String sql; + protected StageGroup stageGroup; + /** + * 前置指纹记录 + */ + protected transient PreFingerprint preFingerprint = null; + + //监控信息 + protected transient StageMetric stageMetric=new StageMetric(); + public AbstractStage() { setType(TYPE); } - protected transient AtomicLong TOTAL = new AtomicLong(0); - protected transient AtomicLong FILTER = new AtomicLong(0); - protected transient Long lastUpdateTime = null; - - @Override - public T doMessage(T t, AbstractContext context) { + @Override public T doMessage(T t, AbstractContext context) { + long startTime=stageMetric.startCalculate(t); try { TraceUtil.debug(t.getHeader().getTraceId(), "AbstractStage", label, t.getMessageBody().toJSONString()); } catch (Exception e) { LOG.error("t.getMessageBody() parse error", e); } - IStageHandle handle = selectHandle(t, context); + IStageHandle handle = selectHandle(t, context); if (handle == null) { return t; } Object result = handle.doMessage(t, context); - // + stageMetric.endCalculate(startTime); if (!context.isContinue() || result == null) { + if(context.getNotFireReason()!=null){ + stageMetric.filterCalculate(context.getNotFireReason()); + } return (T) context.breakExecute(); } + stageMetric.outCalculate(); + context.removeNotFireReason(); return (T) result; } @@ -118,8 +135,7 @@ public T doMessage(T t, AbstractContext context) { * * @return */ - // public abstract AbstractStage copy(); - protected abstract IStageHandle selectHandle(T t, AbstractContext context); + protected abstract IStageHandle selectHandle(T t, AbstractContext context); public String getName() { return name; @@ -160,42 +176,46 @@ public void setLabel(String label) { } public List doRoute(T t) { - String routeLabel = t.getHeader().getRouteLables(); - String filterLabel = t.getHeader().getFilterLables(); + String routeLabel = t.getHeader().getRouteLabels(); + String filterLabel = t.getHeader().getFilterLabels(); + t.getHeader().setRouteLabels(null); + t.getHeader().setFilterLabels(null); if (StringUtil.isEmpty(routeLabel) && StringUtil.isEmpty(filterLabel)) { return this.nextStageLabels; } - List lables = new ArrayList<>(); - lables.addAll(this.nextStageLabels); + + + List labels = new ArrayList<>(this.nextStageLabels); if (StringUtil.isNotEmpty(routeLabel)) { - lables = new ArrayList<>(); + Set routeLabelSet=t.getHeader().createRouteLableSet(routeLabel); + labels = new ArrayList<>(); for (String tempLabel : this.nextStageLabels) { - if (tempLabel != null && routeLabel.indexOf(tempLabel) != -1) { - lables.add(tempLabel); + if (routeLabelSet.contains(tempLabel)) { + labels.add(tempLabel); } } } - if (StringUtil.isNotEmpty(filterLabel)) { + Set routeFilterLabelSet=t.getHeader().createRouteLableSet(filterLabel); for (String tempLabel : this.nextStageLabels) { - if (label != null && filterLabel.indexOf(label) != 1) { - lables.remove(tempLabel); + if (routeFilterLabelSet.contains(label)) { + labels.remove(tempLabel); } } } - return lables; + return labels; } /** * 从配置文件加载日志指纹信息,如果存在做指纹优化 */ protected PreFingerprint loadLogFinger() { - ChainPipeline pipline = (ChainPipeline) getPipeline(); + ChainPipeline pipeline = (ChainPipeline) getPipeline(); String filterName = getLabel(); - if (pipline.isTopology() == false) { - List stages = pipline.getStages(); + if (!pipeline.isTopology()) { + List> stages = pipeline.getStages(); int i = 0; - for (AbstractStage stage : stages) { + for (AbstractStage stage : stages) { if (stage == this) { break; } @@ -203,16 +223,16 @@ protected PreFingerprint loadLogFinger() { } filterName = i + ""; } - String stageIdentification = MapKeyUtil.createKeyBySign(".", pipline.getNameSpace(), pipline.getConfigureName(), filterName); + String stageIdentification = MapKeyUtil.createKeyBySign(".", pipeline.getNameSpace(), pipeline.getConfigureName(), filterName); if (this.filterFieldNames == null) { this.filterFieldNames = ComponentCreator.getProperties().getProperty(stageIdentification); } if (this.filterFieldNames == null) { return null; } - PreFingerprint preFingerprint = createPreFinerprint(stageIdentification); + PreFingerprint preFingerprint = createPreFingerprint(stageIdentification); if (preFingerprint != null) { - pipline.registPreFingerprint(preFingerprint); + pipeline.registPreFingerprint(preFingerprint); } return preFingerprint; } @@ -223,59 +243,56 @@ protected PreFingerprint loadLogFinger() { * @return */ - protected PreFingerprint createPreFinerprint(String stageIdentification) { - ChainPipeline pipline = (ChainPipeline) getPipeline(); - String sourceLable = null; - String nextLable = null; - if (pipline.isTopology()) { - Map stageMap = pipline.createStageMap(); - AbstractStage currentStage = this; - List prewLables = currentStage.getPrevStageLabels(); - while (prewLables != null && prewLables.size() > 0) { - if (prewLables.size() > 1) {//union - sourceLable = null; - nextLable = null; + protected PreFingerprint createPreFingerprint(String stageIdentification) { + ChainPipeline pipeline = (ChainPipeline) getPipeline(); + String sourceLabel = null; + String nextLabel = null; + if (pipeline.isTopology()) { + Map> stageMap = pipeline.createStageMap(); + AbstractStage currentStage = this; + List preLabels = currentStage.getPrevStageLabels(); + while (preLabels != null && preLabels.size() > 0) { + if (preLabels.size() > 1) {//union + sourceLabel = null; + nextLabel = null; break; } - String lable = prewLables.get(0); - AbstractStage stage = (AbstractStage) stageMap.get(lable); + String lable = preLabels.get(0); + AbstractStage stage = stageMap.get(lable); if (stage != null) { if (stage.isAsyncNode()) {//window (join,Statistics) - sourceLable = null; - nextLable = null; + sourceLabel = null; + nextLabel = null; break; } - nextLable = currentStage.getLabel(); + nextLabel = currentStage.getLabel(); currentStage = stage; - sourceLable = currentStage.getLabel(); + sourceLabel = currentStage.getLabel(); if (stage.getNextStageLabels() != null && stage.getNextStageLabels().size() > 1) { break; } } else { - sourceLable = pipline.getChannelName(); - nextLable = currentStage.getLabel(); + sourceLabel = pipeline.getChannelName(); + nextLabel = currentStage.getLabel(); break; } - prewLables = currentStage.getPrevStageLabels(); + preLabels = currentStage.getPrevStageLabels(); } - if (prewLables == null || prewLables.size() == 0) { - sourceLable = pipline.getChannelName(); - nextLable = currentStage.getLabel(); + if (preLabels == null || preLabels.size() == 0) { + sourceLabel = pipeline.getChannelName(); + nextLabel = currentStage.getLabel(); } - if (sourceLable == null || nextLable == null) { + if (sourceLabel == null || nextLabel == null) { return null; } - PreFingerprint preFingerprint = new PreFingerprint(this.filterFieldNames, stageIdentification, sourceLable, nextLable, -1, this, FingerprintCache.getInstance()); - return preFingerprint; + return new PreFingerprint(this.filterFieldNames, stageIdentification, sourceLabel, nextLabel, -1, this, FingerprintCache.getInstance()); } else { - PreFingerprint preFingerprint = new PreFingerprint(this.filterFieldNames, stageIdentification, "0", "0", -1, this, FingerprintCache.getInstance()); - return preFingerprint; + return new PreFingerprint(this.filterFieldNames, stageIdentification, "0", "0", -1, this, FingerprintCache.getInstance()); } } - @Override - public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { + @Override public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { } @@ -299,11 +316,11 @@ public void setMsgSourceName(String msgSourceName) { this.msgSourceName = msgSourceName; } - public Pipeline getPipeline() { + public Pipeline getPipeline() { return pipeline; } - public void setPipeline(Pipeline pipeline) { + public void setPipeline(Pipeline pipeline) { this.pipeline = pipeline; } @@ -315,6 +332,18 @@ public void setOwnerSqlNodeTableName(String ownerSqlNodeTableName) { this.ownerSqlNodeTableName = ownerSqlNodeTableName; } + public StageMetric getStageMetric() { + return stageMetric; + } + + public String getSql() { + return sql; + } + + public void setSql(String sql) { + this.sql = sql; + } + public void setNextStageLabels(List nextStageLabels) { this.nextStageLabels = nextStageLabels; } @@ -326,4 +355,20 @@ public String getFilterFieldNames() { public void setFilterFieldNames(String filterFieldNames) { this.filterFieldNames = filterFieldNames; } + + public PreFingerprint getPreFingerprint() { + return preFingerprint; + } + + public void setPreFingerprint(PreFingerprint preFingerprint) { + this.preFingerprint = preFingerprint; + } + + public StageGroup getStageGroup() { + return stageGroup; + } + + public void setStageGroup(StageGroup stageGroup) { + this.stageGroup = stageGroup; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IWindow.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IWindow.java index 7efd0067..b04d894d 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IWindow.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IWindow.java @@ -23,12 +23,13 @@ import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; import org.apache.rocketmq.streams.common.topology.SectionPipeline; +import org.apache.rocketmq.streams.common.topology.shuffle.IShuffleKeyGenerator; /** * Window Definition 处理数据并输出 */ public interface IWindow - extends IStreamOperator>, IConfigurable { + extends IStreamOperator>, IConfigurable, IShuffleKeyGenerator { int DEFAULTFIRE_MODE = 0;// fire time=endtime+watermark int MULTI_WINDOW_INSTANCE_MODE = 1;// fire at window size interval, until event time >endtime+watermark, every window result is independent diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Pipeline.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Pipeline.java index cad6ebee..c79c18ee 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Pipeline.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/Pipeline.java @@ -34,6 +34,7 @@ import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; import org.apache.rocketmq.streams.common.optimization.MessageGlobleTrace; import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.utils.StringUtil; /** @@ -55,14 +56,19 @@ public class Pipeline extends BasedConfigurable implements I /** * stage列表 */ - protected List stages = new ArrayList<>(); + protected List> stages = new ArrayList<>(); - //给数据源取个名字,主要用于同源任务归并 - private String sourceIdentification;//数据源名称,如果pipline是数据源pipline需要设置 - protected String msgSourceName;//主要用于在join,union场景,标记上游节点用 + /** + * 给数据源取个名字,主要用于同源任务归并,数据源名称,如果pipeline是数据源pipeline需要设置 + */ + private String sourceIdentification; + /** + * 主要用于在join,union场景,标记上游节点用 + */ + protected String msgSourceName; /** - * KEY: source stage lable value: key:next stage lable: value :PreFingerprint + * KEY: source stage label value: key:next stage label: value :PreFingerprint */ protected transient Map> preFingerprintExecutor = new HashMap<>(); @@ -70,8 +76,7 @@ public Pipeline() { setType(TYPE); } - @Override - public T doMessage(T t, AbstractContext context) { + @Override public T doMessage(T t, AbstractContext context) { T message = doMessage(t, context, null); return message; } @@ -90,16 +95,16 @@ public T doMessage(T t, AbstractContext context, AbstractStage... replaceStage) * @param replaceStage * @return */ - protected T doMessageInner(T t, AbstractContext context, AbstractStage... replaceStage) { + protected T doMessageInner(T t, AbstractContext context, AbstractStage... replaceStage) { return doMessageFromIndex(t, context, 0, replaceStage); } - public T doMessageFromIndex(T t, AbstractContext context, int index, AbstractStage... replaceStage) { + public T doMessageFromIndex(T t, AbstractContext context, int index, AbstractStage... replaceStage) { context.setMessage(t); //boolean needFlush = needFlush(t); for (int i = index; i < stages.size(); i++) { - AbstractStage oriStage = stages.get(i); - AbstractStage stage = chooseReplaceStage(oriStage, replaceStage); + AbstractStage oriStage = stages.get(i); + AbstractStage stage = chooseReplaceStage(oriStage, replaceStage); boolean isContinue = executeStage(stage, t, context); if (!isContinue) { if (stage.isAsyncNode()) { @@ -113,6 +118,13 @@ public T doMessageFromIndex(T t, AbstractContext context, int index, AbstractSta return t; } + @Override protected boolean initConfigurable() { + for(AbstractStage stage:stages){ + stage.init(); + } + return super.initConfigurable(); + } + /** * regist pre filter Fingerprint * @@ -122,12 +134,8 @@ protected void registPreFingerprint(PreFingerprint preFingerprint) { if (preFingerprint == null) { return; } - Map preFingerprintMap = this.preFingerprintExecutor.get(preFingerprint.getSourceStageLable()); - if (preFingerprintMap == null) { - preFingerprintMap = new HashMap<>(); - this.preFingerprintExecutor.put(preFingerprint.getSourceStageLable(), preFingerprintMap); - } - preFingerprintMap.put(preFingerprint.getNextStageLable(), preFingerprint); + Map preFingerprintMap = this.preFingerprintExecutor.computeIfAbsent(preFingerprint.getSourceStageLabel(), k -> new HashMap<>()); + preFingerprintMap.put(preFingerprint.getNextStageLabel(), preFingerprint); } protected PreFingerprint getPreFingerprint(String currentLable, String nextLable) { @@ -176,7 +184,12 @@ protected boolean executeStage(AbstractStage stage, T t, AbstractContext context T lastMsg = null; for (T subT : oldSplits) { context.closeSplitMode(subT); - subT.getHeader().setMsgRouteFromLable(t.getHeader().getMsgRouteFromLable()); + if(ChainPipeline.class.isInstance(this)&&!((ChainPipeline)this).isTopology()&&StringUtil.isNotEmpty(this.msgSourceName)){ + subT.getHeader().setMsgRouteFromLable(t.getHeader().getMsgRouteFromLable()); + }else { + subT.getHeader().setMsgRouteFromLable(t.getHeader().getMsgRouteFromLable()); + } + subT.getHeader().addLayerOffset(splitMessageOffset); splitMessageOffset++; boolean isContinue = doMessage(subT, stage, context); @@ -287,12 +300,11 @@ public void setStageLable(AbstractStage stage, String lable) { stage.setLabel(lable); } - public void setStages(List stages) { + public void setStages(List> stages) { this.stages = stages; } - @Override - public void destroy() { + @Override public void destroy() { if (LOG.isInfoEnabled()) { LOG.info(getName() + " is destroy, release pipline " + stages.size()); } @@ -307,7 +319,7 @@ public void setName(String name) { this.name = name; } - public List getStages() { + public List> getStages() { return stages; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/PipelineSourceJoiner.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/PipelineSourceJoiner.java deleted file mode 100644 index 0e34a8b8..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/PipelineSourceJoiner.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.streams.common.topology.model; - -import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; - -/** - * 帮助某个pipline 关联对应的数据源pipline。 主要用于多个同源任务自动装配的场景 namespace:source pipline的name space name:piplineName - */ -public class PipelineSourceJoiner extends BasedConfigurable { - public static String TYPE = "joiner"; - protected String sourcePipelineName;//数据源对应的pipline name - protected String pipelineName;//需要关联的pipline name - - public PipelineSourceJoiner() { - setType(TYPE); - } - - public String getSourcePipelineName() { - return sourcePipelineName; - } - - public void setSourcePipelineName(String sourcePipelineName) { - this.sourcePipelineName = sourcePipelineName; - } - - public String getPipelineName() { - return pipelineName; - } - - public void setPipelineName(String pipelineName) { - this.pipelineName = pipelineName; - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/IShuffleKeyGenerator.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/IShuffleKeyGenerator.java new file mode 100644 index 00000000..d5b613c0 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/IShuffleKeyGenerator.java @@ -0,0 +1,24 @@ +/* + * 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.rocketmq.streams.common.topology.shuffle; + +import org.apache.rocketmq.streams.common.context.IMessage; + +public interface IShuffleKeyGenerator { + + String generateShuffleKey(IMessage message); +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/ShuffleMQCreator.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/ShuffleMQCreator.java new file mode 100644 index 00000000..68bd088d --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/shuffle/ShuffleMQCreator.java @@ -0,0 +1,406 @@ +/* + * 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.rocketmq.streams.common.topology.shuffle; + +import java.lang.reflect.Field; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +import java.util.concurrent.ConcurrentHashMap; +import org.apache.rocketmq.streams.common.cache.softreference.ICache; +import org.apache.rocketmq.streams.common.cache.softreference.impl.SoftReferenceCache; +import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; +import org.apache.rocketmq.streams.common.channel.builder.IShuffleChannelBuilder; +import org.apache.rocketmq.streams.common.channel.impl.memory.MemoryCache; +import org.apache.rocketmq.streams.common.channel.impl.memory.MemoryChannel; +import org.apache.rocketmq.streams.common.channel.impl.memory.MemorySink; +import org.apache.rocketmq.streams.common.channel.impl.memory.MemorySource; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink; +import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.channel.source.AbstractSource; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.apache.rocketmq.streams.common.utils.ServiceLoadUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; + +/** + * create shuffle source and producer + */ +public class ShuffleMQCreator { + private static ICache cache=new SoftReferenceCache<>(); + public static String SHUFFLE_BUILTIN="built-in"; + /** + * input parameters + */ + protected String namespace; + protected String pipelineName; + protected String shuffleOwnerName; + protected int splitCount; + protected transient ISource pipelineSource; + + protected transient List> queueList;//shuffle mq's split list + protected transient Map> queueMap = new ConcurrentHashMap<>(); + + + /** + * need create + */ + protected transient AbstractSupportShuffleSink producer; + protected ISource consumer; + + /** + * + */ + protected volatile boolean hasCreateShuffleChannel = false; + + private ShuffleMQCreator(ISource source,String namespace,String pipelineName,String shuffleOwnerName, int splitCount){ + this.pipelineSource=source; + this.namespace=namespace; + this.pipelineName=pipelineName; + this.shuffleOwnerName=shuffleOwnerName; + this.splitCount=splitCount; + } + + public static ShuffleMQCreator createShuffleCreator(ISource source,String namespace,String pipelineName,String shuffleOwnerName, int splitCount){ + String key= MapKeyUtil.createKey(namespace,pipelineName,shuffleOwnerName); + ShuffleMQCreator shuffleMQCreator=cache.get(key); + if(shuffleMQCreator!=null){ + return shuffleMQCreator; + } + shuffleMQCreator=new ShuffleMQCreator(source,namespace,pipelineName,shuffleOwnerName,splitCount); + shuffleMQCreator.init(); + cache.put(key,shuffleMQCreator); + return shuffleMQCreator; + } + + + public static ISource getSource(String namespace,String pipelineName,String shuffleOwnerName ){ + String key= MapKeyUtil.createKey(namespace,pipelineName,shuffleOwnerName); + ShuffleMQCreator shuffleMQCreator=cache.get(key); + if(shuffleMQCreator==null){ + throw new RuntimeException("expect get ShuffleMQCreator in cache, but not. Check whether it is created in the shuffleproducerchainstage class "); + } + return shuffleMQCreator.getConsumer(); + } + + + + + + /** + * init shuffle channel + */ + private void init() { + String channelType =this.pipelineSource.getClass().getSimpleName(); + if(SHUFFLE_BUILTIN.equals(channelType)){ + ISource source=createSourceByProperty(namespace, pipelineName); + source.setNameSpace(namespace); + autoCreateShuffleChannel(source); + }else{ + consumer= createSourceByProperty(namespace, pipelineName); + producer = createSinkByProperty(namespace, pipelineName); + if (consumer == null || producer== null) { + autoCreateShuffleChannel(pipelineSource); + } + if (consumer == null) { + return; + } + if (consumer instanceof AbstractSource) { + ((AbstractSource) consumer).setJsonData(true); + } + if(producer!=null){ + producer.init(); + } + if (producer != null && (queueList == null || queueList.size() == 0)) { + this.producer.init(); + queueList = producer.getSplitList(); + Map> tmp = new ConcurrentHashMap<>(); + for (ISplit queue : queueList) { + tmp.put(queue.getQueueId(), queue); + } + this.queueMap = tmp; + } + } + + + } + /** + * choose shuffle split + * @param key + * @return + */ + public int hash(Object key) { + int mValue = queueList.size(); + int h = 0; + if (key != null) { + h = key.hashCode() ^ (h >>> 16); + if (h < 0) { + h = -h; + } + } + return h % mValue; + } + + + /** + * 创建channel,根据配置文件配置channel的连接信息 + * + * @return + */ + protected ISource createSourceByProperty(String namespace, String name) { + IChannelBuilder builder = createBuilder(); + if (builder == null) { + return null; + } + Properties properties = createChannelProperties(namespace,this.shuffleOwnerName); + ISource source = builder.createSource(namespace, name, properties, null); + if (source instanceof MemorySource) { + MemorySource memorySource = (MemorySource) source; + MemoryCache memoryCache = new MemoryCache(); + memorySource.setMemoryCache(memoryCache); + memoryCache.init(); + } + source.init(); + return source; + } + + + /** + * 创建channel,根据配置文件配置channel的连接信息 + * + * @return + */ + protected AbstractSupportShuffleSink createSinkByProperty(String namespace, String name) { + + IChannelBuilder builder = createBuilder(); + if (builder == null) { + return null; + } + Properties properties = createChannelProperties(namespace,this.shuffleOwnerName); + + ISink sink = builder.createSink(namespace, name, properties, null); + if (!(sink instanceof AbstractSupportShuffleSink)) { + throw new RuntimeException("can not support shuffle " + sink.toJson()); + } + if (sink instanceof MemorySink) { + MemorySink memorySink = (MemorySink) sink; + if (!(this.consumer instanceof MemorySource)) { + throw new RuntimeException("shuffle cosumer need memory, real is " + this.consumer); + } + MemorySource memorySource = (MemorySource) this.consumer; + MemoryCache memoryCache = memorySource.getMemoryCache(); + memorySink.setMemoryCache(memoryCache); + } + + sink.init(); + return (AbstractSupportShuffleSink) sink; + } + + + /** + * 如果用户未配置shuffle channel,根据pipeline数据源动态创建 + * + */ + public void autoCreateShuffleChannel(ISource pipelineSource) { + if (!hasCreateShuffleChannel) { + synchronized (this) { + if (!hasCreateShuffleChannel) { + String channelType =this.pipelineSource.getClass().getSimpleName(); + IChannelBuilder builder = ServiceLoadUtil.loadService(IChannelBuilder.class,channelType); + if (builder == null) { + throw new RuntimeException("can not create shuffle channel, not find channel builder by the type" + channelType); + } + if (!(builder instanceof IShuffleChannelBuilder)) { + throw new RuntimeException("can not create shuffle channel, builder not impl IShuffleChannelBuilder " + channelType); + } + IShuffleChannelBuilder shuffleChannelBuilder = (IShuffleChannelBuilder) builder; + ISink sink = shuffleChannelBuilder.createBySource(pipelineSource); + sink.init(); + if (!(sink instanceof MemoryChannel) && !(sink instanceof AbstractSupportShuffleSink)) { + throw new RuntimeException("can not create shuffle channel, sink not extends AbstractSupportShuffleSink " + channelType); + } + ISource source = null; + if (sink instanceof MemoryChannel) { + MemoryCache memoryCache = new MemoryCache(); + memoryCache.setNameSpace(namespace); + memoryCache.setConfigureName(shuffleOwnerName); + + sink = new MemorySink(); + source = new MemorySource(); + ((MemorySink) sink).setMemoryCache(memoryCache); + ((MemorySource) source).setMemoryCache(memoryCache); + memoryCache.init(); + } + + Properties properties = new Properties(); + + properties.put("groupName", shuffleOwnerName); + properties.put("tags", shuffleOwnerName); + + AbstractSupportShuffleSink shuffleSink = (AbstractSupportShuffleSink) sink; + shuffleSink.setSplitNum(getShuffleSplitCount(shuffleSink)); + shuffleSink.setNameSpace(namespace); + shuffleSink.setConfigureName(shuffleOwnerName); + String topicFiledName = shuffleSink.getShuffleTopicFieldName(); + String shuffleTopic = null; + //内存模式,是无topic的 + if (StringUtil.isNotEmpty(topicFiledName)) { + String topic = ReflectUtil.getDeclaredField(shuffleSink, topicFiledName); + shuffleTopic = createShuffleTopic(topic); + ReflectUtil.setBeanFieldValue(shuffleSink, topicFiledName, shuffleTopic); + } + + //修改和window有关的属性,如groupname,tags + List fields = ReflectUtil.getDeclaredFieldsContainsParentClass(sink.getClass()); + for (Field field : fields) { + String fieldName = field.getName(); + String value = properties.getProperty(fieldName); + if (StringUtil.isNotEmpty(value)) { + ReflectUtil.setBeanFieldValue(sink, fieldName, value); + } + } + shuffleSink.setHasInit(false); + shuffleSink.init();//在这里完成shuffle channel的创建 + if (source == null) { + source = shuffleChannelBuilder.copy(pipelineSource); + } + + //修改和window有关的属性,如groupname,tags + + fields = ReflectUtil.getDeclaredFieldsContainsParentClass(source.getClass()); + for (Field field : fields) { + String fieldName = field.getName(); + String value = properties.getProperty(fieldName); + if (StringUtil.isNotEmpty(value)) { + ReflectUtil.setBeanFieldValue(source, fieldName, value); + } + } + + source.setNameSpace(sink.getNameSpace()); + source.setConfigureName(sink.getConfigureName()); + //修改主题 + if (shuffleTopic != null && topicFiledName != null) { + ReflectUtil.setBeanFieldValue(source, topicFiledName, shuffleTopic); + } + if (source instanceof AbstractSource) { + AbstractSource abstractSource = (AbstractSource) source; + abstractSource.setHasInit(false); + } + source.init(); + + this.producer = shuffleSink; + this.consumer = source; + } + } + } + } + /** + * create channel builder + * + * @return + */ + protected IChannelBuilder createBuilder() { + String type = ComponentCreator.getProperties().getProperty(ConfigureFileKey.WINDOW_SHUFFLE_CHANNEL_TYPE); + if (StringUtil.isEmpty(type)) { + return null; + } + return ServiceLoadUtil.loadService(IChannelBuilder.class,type); + } + + + /** + * 根据属性文件配置 + * + * @return 资源文件 + */ + protected Properties createChannelProperties(String namespace,String shuffleOwnerName) { + Properties properties = new Properties(); + for (Map.Entry entry : ComponentCreator.getProperties().entrySet()) { + String key = (String) entry.getKey(); + String value = (String) entry.getValue(); + if (key.startsWith(ConfigureFileKey.WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX)) { + String channelKey = key.replace(ConfigureFileKey.WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX, ""); + if (channelKey.startsWith(namespace)) {//支持基于namespace 做shuffle window共享 + channelKey = channelKey.replace(namespace, ""); + properties.put(channelKey, value); + } else { + if (!properties.containsKey(channelKey)) { + properties.put(channelKey, value); + } + } + } + } + Set multiPropertySet = new HashSet<>(); + String dynamicProperty = properties.getProperty("dynamic.property"); + if (dynamicProperty != null) { + + String dynamicPropertyValue = shuffleOwnerName; + String[] mutilPropertys = dynamicProperty.split(","); + + for (String properyKey : mutilPropertys) { + properties.put(properyKey, dynamicPropertyValue); + multiPropertySet.add(properyKey); + } + + } + String groupName = "groupName"; + if (!multiPropertySet.contains(groupName)) { + properties.put(groupName, shuffleOwnerName); + } + if (!multiPropertySet.contains("tags")) { + properties.put("tags", shuffleOwnerName); + } + return properties; + } + + protected int getShuffleSplitCount(AbstractSupportShuffleSink shuffleSink) { + if(this.splitCount>0){ + return splitCount; + } + int splitNum = shuffleSink.getSplitNum(); + return splitNum > 0 ? splitNum : 32; + } + + public List> getQueueList() { + return queueList; + } + + /** + * 1个pipeline一个 shuffle topic + * + * @param topic + * @return + */ + protected String createShuffleTopic(String topic) { + return "shuffle_" + topic + "_" + namespace.replaceAll("\\.", "_") + "_" + pipelineName.replaceAll("\\.", "_").replaceAll(";", "_"); + } + + public AbstractSupportShuffleSink getProducer() { + return producer; + } + + public ISource getConsumer() { + return consumer; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractWindowStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractWindowStage.java index abe81588..4385855c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractWindowStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/AbstractWindowStage.java @@ -29,6 +29,7 @@ import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.ChainStage; import org.apache.rocketmq.streams.common.topology.model.IWindow; @@ -78,7 +79,9 @@ public void batchMessageFinish(IMessage message, AbstractContext context, BatchF @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { window = configurableService.queryConfigurable(IWindow.TYPE, windowName); - window.setFireReceiver(getReceiverAfterCurrentNode()); + if (((ChainPipeline)getPipeline()).isTopology()) { + window.setFireReceiver(getReceiverAfterCurrentNode()); + } if (Boolean.TRUE.equals(Boolean.valueOf(ComponentCreator.getProperties().getProperty(ConfigureFileKey.DIPPER_RUNNING_STATUS, ConfigureFileKey.DIPPER_RUNNING_STATUS_DEFAULT)))) { window.windowInit(); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/EmptyChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/EmptyChainStage.java new file mode 100644 index 00000000..13b24ed2 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/EmptyChainStage.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.common.topology.stages; + +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.topology.model.IStageHandle; + +public class EmptyChainStage extends AbstractStatelessChainStage { + protected transient IStageHandle handle = new IStageHandle() { + @Override public String getName() { + return getClass().getName(); + } + + @Override + protected IMessage doProcess(IMessage message, AbstractContext context) { + return proccessMessage(message,context); + } + }; + + protected IMessage proccessMessage(IMessage message, AbstractContext context) { + return message; + } + @Override public boolean isAsyncNode() { + return false; + } + + @Override protected IStageHandle selectHandle(T t, AbstractContext context) { + return handle ; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/FilterChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/FilterChainStage.java index ec8a9590..222c4b68 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/FilterChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/FilterChainStage.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.concurrent.atomic.AtomicInteger; +import javax.swing.ImageIcon; import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.component.IComponent; import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; @@ -31,9 +32,11 @@ import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.interfaces.IFilterService; +import org.apache.rocketmq.streams.common.monitor.ConsoleMonitorManager; import org.apache.rocketmq.streams.common.monitor.TopologyFilterMonitor; import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; import org.apache.rocketmq.streams.common.topology.model.AbstractRule; import org.apache.rocketmq.streams.common.topology.model.AbstractStage; import org.apache.rocketmq.streams.common.topology.model.IStageHandle; @@ -47,17 +50,21 @@ public class FilterChainStage extend protected transient AtomicInteger count = new AtomicInteger(0); protected transient Map map = new HashMap<>(); private List names; - protected String nameRegex;//通过名字匹配模式,加载一批规则,避免大批量输入规则名称 + /** + * 通过名字匹配模式,加载一批规则,避免大批量输入规则名称 + */ + protected String nameRegex; private transient List rules; private transient Map ruleName2JsonObject = new HashMap<>(); public static transient Class componentClass = ReflectUtil.forClass("org.apache.rocketmq.streams.filter.FilterComponent"); protected boolean openHyperscan = false; protected static transient IComponent component; - - protected transient PreFingerprint preFingerprint = null; + protected transient FilterChainStage SELF; public FilterChainStage() { + SELF=this; setEntityName("filter"); + } @Override @@ -69,24 +76,30 @@ public boolean isAsyncNode() { @Override protected IMessage doProcess(IMessage message, AbstractContext context) { boolean isTrace = TraceUtil.hit(message.getHeader().getTraceId()); - if (isTrace) { - traceRuleInfo(message); - } + if (component == null) { component = ComponentCreator.getComponent(null, componentClass); } - message.getHeader().setPiplineExecutorMonitor(new TopologyFilterMonitor()); + String fieldValue=message.getHeader().getLogFingerprintValue(); + NotFireReason notFireReason=null; + if(preFingerprint!=null){ + notFireReason=new NotFireReason(SELF,fieldValue); + context.setNotFireReason(notFireReason); + } + List fireRules = component.getService().executeRule(message, context, rules); + //not match rules if (fireRules == null || fireRules.size() == 0) { context.breakExecute(); if (preFingerprint != null) { preFingerprint.addLogFingerprintToSource(message); } - // if(isTrace){ - traceFailExpression(message); - //} + notFireReason=context.getNotFireReason(); + if(isTrace&¬FireReason!=null){ + traceFailExpression(message,notFireReason); + } } return message; } @@ -97,125 +110,11 @@ public String getName() { } }; - protected void traceRuleInfo(IMessage message) { - TopologyFilterMonitor monitor = message.getHeader().getPiplineExecutorMonitor(); - if (monitor != null) { - if (monitor.getNotFireExpression2DependentFields() != null) { - - Map> notFireExpressions = monitor.getNotFireExpression2DependentFields(); - Iterator>> it = notFireExpressions.entrySet().iterator(); - String description = "the View " + getOwnerSqlNodeTableName() + " break ,has " + notFireExpressions.size() + " expression not fire:" + PrintUtil.LINE; - StringBuilder stringBuilder = new StringBuilder(description); - int index = 1; - while (it.hasNext()) { - Entry> entry = it.next(); - String expression = entry.getKey(); - List dependentFields = entry.getValue(); - for (String dependentField : dependentFields) { - List scripts = findScriptByStage(dependentField); - if (scripts != null) { - for (String script : scripts) { - stringBuilder.append(script + PrintUtil.LINE); - } - } - } - stringBuilder.append("The " + index++ + " expression is " + PrintUtil.LINE + getExpressionDescription(expression, message) + PrintUtil.LINE); - } - TraceUtil.debug(message.getHeader().getTraceId(), "break rule", stringBuilder.toString()); - } - - } - } - - protected void traceFailExpression(IMessage message) { - TopologyFilterMonitor monitor = message.getHeader().getPiplineExecutorMonitor(); - if (monitor != null) { - if (monitor.getNotFireExpression2DependentFields() != null) { - - Map> notFireExpressions = monitor.getNotFireExpression2DependentFields(); - Iterator>> it = notFireExpressions.entrySet().iterator(); - String description = "the View " + getOwnerSqlNodeTableName() + " break ,has " + notFireExpressions.size() + " expression not fire:" + PrintUtil.LINE; - StringBuilder stringBuilder = new StringBuilder(description); - int index = 1; - while (it.hasNext()) { - Entry> entry = it.next(); - String expression = entry.getKey(); - List dependentFields = entry.getValue(); - for (String dependentField : dependentFields) { - List scripts = findScriptByStage(dependentField); - if (scripts != null) { - for (String script : scripts) { - stringBuilder.append(script + PrintUtil.LINE); - } - } - } - stringBuilder.append("The " + index++ + " expression is " + PrintUtil.LINE + getExpressionDescription(expression, message) + PrintUtil.LINE); - } - TraceUtil.debug(message.getHeader().getTraceId(), "break rule", stringBuilder.toString()); - } - - } - } - - /** - * 如果是表达式,把表达式的值也提取出来 - * - * @param expression - * @param message - * @return - */ - protected String getExpressionDescription(String expression, IMessage message) { - if (expression.startsWith("(")) { - int index = expression.indexOf(","); - String varName = expression.substring(1, index); - String value = message.getMessageBody().getString(varName); - return expression + ", the " + varName + " is " + value; - } - return expression; - } - - protected List findScriptByStage(String notFireBooleanVar) { - if (notFireBooleanVar == null || !notFireBooleanVar.startsWith("__")) { - return null; - } - ScriptChainStage stage = findScriptChainStage(this); - return stage.getDependentScripts(notFireBooleanVar); + protected void traceFailExpression(IMessage message,NotFireReason notFireReason) { + ConsoleMonitorManager.getInstance().reportOutput(FilterChainStage.this, message, ConsoleMonitorManager.MSG_FILTERED, notFireReason.toString()); + TraceUtil.debug(message.getHeader().getTraceId(), "break rule", notFireReason.toString()); } - protected ScriptChainStage findScriptChainStage(AbstractStage stage) { - ChainPipeline pipline = (ChainPipeline) stage.getPipeline(); - if (pipline.isTopology()) { - List lableNames = stage.getPrevStageLabels(); - if (lableNames != null) { - for (String lableName : lableNames) { - Map stageMap = pipline.getStageMap(); - AbstractStage prewStage = stageMap.get(lableName); - if (prewStage != null && ScriptChainStage.class.isInstance(prewStage)) { - return (ScriptChainStage) prewStage; - } - if (prewStage != null) { - return findScriptChainStage(prewStage); - } - } - } - return null; - } else { - List stages = pipline.getStages(); - int i = 0; - for (; i < stages.size(); i++) { - if (stages.get(i).equals(stage)) { - break; - } - } - for (; i >= 0; i--) { - AbstractStage prewStage = stages.get(i); - if (prewStage instanceof ScriptChainStage) { - return (ScriptChainStage) prewStage; - } - } - return null; - } - } @Override protected IStageHandle selectHandle(T t, AbstractContext context) { @@ -253,6 +152,9 @@ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableS String filterName = getLabel(); for (String name : names) { AbstractRule rule = configurableService.queryConfigurable(AbstractRule.TYPE, name); + if(rule==null){ + throw new RuntimeException("the rule expect exist, but not. the rule name is "+name); + } rules.add((R) rule); if (!this.isOpenHyperscan()) { /** @@ -324,6 +226,7 @@ public void setOpenHyperscan(boolean openHyperscan) { this.openHyperscan = openHyperscan; } + @Override public void setPreFingerprint(PreFingerprint preFingerprint) { this.preFingerprint = preFingerprint; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinChainStage.java index 811f157f..9cb9df35 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinChainStage.java @@ -27,8 +27,9 @@ import org.apache.rocketmq.streams.common.topology.model.Pipeline; /** - * 会在解析时,增加一个script(_join_flag='true'),在rigth/left pipline加一个window对应的stage + * 新的解析已经废弃,主要兼容老的规则数据 */ +@Deprecated public class JoinChainStage extends AbstractWindowStage { protected String leftPipelineName; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinEndChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinEndChainStage.java new file mode 100644 index 00000000..c88d20a7 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinEndChainStage.java @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.common.topology.stages; + +public class JoinEndChainStage extends EmptyChainStage { +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinStartChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinStartChainStage.java new file mode 100644 index 00000000..c7422ee8 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/JoinStartChainStage.java @@ -0,0 +1,67 @@ +/* + * 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.rocketmq.streams.common.topology.stages; + +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.context.MessageHeader; +import org.apache.rocketmq.streams.common.topology.model.IStageHandle; + +public class JoinStartChainStage extends EmptyChainStage { + protected String rightDependentTableName; + protected String leftLableName; + protected String rightLableName; + @Override + protected IMessage proccessMessage(IMessage message, AbstractContext context) { + String lable = message.getHeader().getMsgRouteFromLable(); + if (lable != null) { + if (lable.equals(rightDependentTableName)) { + message.getHeader().addRouteLabel(rightLableName); + } else { + message.getHeader().addRouteLabel(leftLableName); + } + + } else { + throw new RuntimeException("can not dipatch message, need route label " + toJson()); + } + return message; + } + + public String getRightDependentTableName() { + return rightDependentTableName; + } + + public void setRightDependentTableName(String rightDependentTableName) { + this.rightDependentTableName = rightDependentTableName; + } + + public String getLeftLableName() { + return leftLableName; + } + + public void setLeftLableName(String leftLableName) { + this.leftLableName = leftLableName; + } + + public String getRightLableName() { + return rightLableName; + } + + public void setRightLableName(String rightLableName) { + this.rightLableName = rightLableName; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OutputChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OutputChainStage.java index d9c7c7fa..05f18833 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OutputChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/OutputChainStage.java @@ -19,6 +19,7 @@ import java.util.HashSet; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; import org.apache.rocketmq.streams.common.channel.IChannel; import org.apache.rocketmq.streams.common.channel.sink.ISink; import org.apache.rocketmq.streams.common.channel.source.systemmsg.NewSplitMessage; @@ -61,45 +62,50 @@ public class OutputChainStage extends ChainStage implemen protected transient IStageHandle handle = new IStageHandle() { @Override protected IMessage doProcess(IMessage message, AbstractContext context) { - if (StringUtil.isNotEmpty(closeOutput)) { - String tmp = closeOutput.toLowerCase(); - if ("true".equals(tmp) || "false".equals(tmp)) { - Boolean value = Boolean.valueOf(tmp); - if (value) { + return doSink(message,context); + } + + @Override + public String getName() { + return OutputChainStage.class.getName(); + } + }; + + + protected IMessage doSink(IMessage message, AbstractContext context){ + if (StringUtil.isNotEmpty(closeOutput)) { + String tmp = closeOutput.toLowerCase(); + if ("true".equals(tmp) || "false".equals(tmp)) { + Boolean value = Boolean.valueOf(tmp); + if (value) { + return message; + } + } else { + tmp = getENVVar(closeOutput); + if (StringUtil.isNotEmpty(tmp)) { + if ("true".equals(tmp.toLowerCase())) { return message; } - } else { - tmp = getENVVar(closeOutput); - if (StringUtil.isNotEmpty(tmp)) { - if ("true".equals(tmp.toLowerCase())) { - return message; - } - } } } - boolean isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest"); - if (isWindowTest) { - System.out.println("output count is " + count.incrementAndGet()); - } - /** - * 主要是输出可能影响线上数据,可以通过配置文件的开关,把所有的输出,都指定到一个其他输出中 - */ - if (openMockChannel()) { - if (mockSink != null) { - mockSink.batchAdd(message); - return message; - } + } + boolean isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest"); + if (isWindowTest) { + System.out.println("output count is " + count.incrementAndGet()); + } + /** + * 主要是输出可能影响线上数据,可以通过配置文件的开关,把所有的输出,都指定到一个其他输出中 + */ + if (openMockChannel()) { + if (mockSink != null) { + mockSink.batchAdd(message); return message; } - sink.batchAdd(message); return message; } - - @Override - public String getName() { - return OutputChainStage.class.getName(); - } - }; + sink.batchAdd(message); + return message; + } @Override public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { @@ -136,6 +142,17 @@ public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMe } + + @Override public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { + ISink realSink = null; + if (openMockChannel() && mockSink != null) { + realSink = mockSink; + } else { + realSink = sink; + } + realSink.flush(); + + } @Override protected IStageHandle selectHandle(T t, AbstractContext context) { return handle; @@ -195,7 +212,7 @@ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableS mockSink = getMockChannel(configurableService, sink.getNameSpace()); } - private ISink getMockChannel(IConfigurableService configurableService, String nameSpace) { + protected ISink getMockChannel(IConfigurableService configurableService, String nameSpace) { String type = ComponentCreator.getProperties().getProperty("out.mock.type"); if (type == null) { return null; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleConsumerChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleConsumerChainStage.java new file mode 100644 index 00000000..af1d6109 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleConsumerChainStage.java @@ -0,0 +1,193 @@ +/* + * 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.rocketmq.streams.common.topology.stages; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.channel.source.systemmsg.NewSplitMessage; +import org.apache.rocketmq.streams.common.channel.source.systemmsg.RemoveSplitMessage; +import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage; +import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; +import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.Context; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; +import org.apache.rocketmq.streams.common.interfaces.ISystemMessage; +import org.apache.rocketmq.streams.common.topology.SectionPipeline; +import org.apache.rocketmq.streams.common.topology.model.AbstractRule; +import org.apache.rocketmq.streams.common.topology.model.IStageHandle; +import org.apache.rocketmq.streams.common.topology.shuffle.ShuffleMQCreator; +import org.apache.rocketmq.streams.common.utils.CompressUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.common.utils.TraceUtil; + +public class ShuffleConsumerChainStage extends AbstractStatelessChainStage implements IAfterConfigurableRefreshListener { + /** + * 消息所属的window + */ + protected transient String MSG_OWNER = "MSG_OWNER"; + protected static final String SHUFFLE_MESSAGES = "SHUFFLE_MESSAGES"; + private static final String SHUFFLE_TRACE_ID = "SHUFFLE_TRACE_ID"; + public static final String SHUFFLE_OFFSET = "SHUFFLE_OFFSET"; + + + private transient SectionPipeline lastStages; + + + protected transient ISource consumer; + protected String shuffleOwnerName;//shuffle 拥有者到名子,如是窗口,则是windowname+groupname+updateflag + + + + + protected transient AtomicBoolean hasStart = new AtomicBoolean(false); + public void startSource() { + if (consumer == null) { + return; + } + if (hasStart.compareAndSet(false, true)) { + consumer.start(new IStreamOperator() { + @Override public Object doMessage(IMessage message, AbstractContext context) { + sendMessage(message,context); + return message; + } + }); + } + + } + + + /** + * 接收到分片信息,如果是系统消息,做缓存刷新,否则把消息放入缓存,同时计算存储的有效性 + * + * @param oriMessage + * @param context + * @return + */ + + protected transient AtomicLong COUNT = new AtomicLong(0); + + /** + * send shuffle msg 2 last node + * @param oriMessage + * @param context + */ + protected void sendMessage(IMessage oriMessage, AbstractContext context) { + if (oriMessage.getHeader().isSystemMessage()) { + lastStages.doMessage(oriMessage,context); + return ; + + } + if (oriMessage.getMessageBody().getBooleanValue(ShuffleProducerChainStage.IS_COMPRESSION_MSG)) { + byte[] bytes = oriMessage.getMessageBody().getBytes(ShuffleProducerChainStage.COMPRESSION_MSG_DATA); + String msgStr = CompressUtil.unGzip(bytes); + oriMessage.setMessageBody(JSONObject.parseObject(msgStr)); + } + /** + * 过滤不是这个window的消息,一个shuffle通道,可能多个window共享,这里过滤掉非本window的消息 + */ + boolean isFilter = filterNotOwnerMessage(oriMessage); + if (isFilter) { + return; + } + String queueId = oriMessage.getHeader().getQueueId(); + JSONArray messages = oriMessage.getMessageBody().getJSONArray(SHUFFLE_MESSAGES); + if (messages == null) { + return; + } + + String traceId = oriMessage.getMessageBody().getString(SHUFFLE_TRACE_ID); + if (!StringUtil.isEmpty(traceId)) { + TraceUtil.debug(traceId, "shuffle message in", "received message size:" + messages.size()); + } + int i=0; + for (Object obj : messages) { + IMessage message = new Message((JSONObject) obj); + message.getHeader().setQueueId(queueId); + message.getHeader().setOffset(oriMessage.getHeader().getOffset()); + message.getHeader().addLayerOffset(i++); + /** + * create new offset + */ + message.getMessageBody().put(SHUFFLE_OFFSET, oriMessage.getHeader().getOffset()); + lastStages.doMessage(message,new Context(message)); + } + } + + /** + * When the producer receives the first piece of data, it will notify the consumer to start consumption + * @param t + * @param context + * @return + */ + @Override protected IStageHandle selectHandle(T t, AbstractContext context) { + return new IStageHandle() { + @Override protected T doProcess(T t, AbstractContext context) { + startSource(); + return null; + } + + @Override public String getName() { + return ShuffleConsumerChainStage.class.getName(); + } + }; + } + /** + * 过滤掉不是这个window的消息 + * + * @param oriMessage + * @return + */ + protected boolean filterNotOwnerMessage(IMessage oriMessage) { + String owner = oriMessage.getMessageBody().getString(MSG_OWNER); + if (owner != null && owner.equals(this.shuffleOwnerName)) { + return false; + } + return true; + } + + + + + @Override public boolean isAsyncNode() { + return false; + } + + public String getShuffleOwnerName() { + return shuffleOwnerName; + } + + public void setShuffleOwnerName(String shuffleOwnerName) { + this.shuffleOwnerName = shuffleOwnerName; + } + + @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { + if(this.consumer==null){ + this.consumer=ShuffleMQCreator.getSource(getPipeline().getNameSpace(),getPipeline().getConfigureName(),shuffleOwnerName); + this.consumer.init(); + lastStages = getReceiverAfterCurrentNode(); + } + + + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleProducerChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleProducerChainStage.java new file mode 100644 index 00000000..b7d759b0 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ShuffleProducerChainStage.java @@ -0,0 +1,345 @@ +/* + * 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.rocketmq.streams.common.topology.stages; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.MutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink; +import org.apache.rocketmq.streams.common.channel.sinkcache.impl.AbstractMultiSplitMessageCache; +import org.apache.rocketmq.streams.common.channel.source.ISource; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.Context; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.model.AbstractRule; +import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.model.IWindow; +import org.apache.rocketmq.streams.common.topology.shuffle.ShuffleMQCreator; +import org.apache.rocketmq.streams.common.utils.CompressUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.common.utils.TraceUtil; + +public class ShuffleProducerChainStage extends OutputChainStage { + private static final Log LOG = LogFactory.getLog(ShuffleProducerChainStage.class); + public static final String IS_COMPRESSION_MSG = "_is_compress_msg"; + public static final String COMPRESSION_MSG_DATA = "_compress_msg"; + public static final String MSG_FROM_SOURCE = "msg_from_source"; + public static final String ORIGIN_OFFSET = "origin_offset"; + + public static final String ORIGIN_QUEUE_ID = "origin_queue_id"; + + public static final String ORIGIN_QUEUE_IS_LONG = "origin_offset_is_LONG"; + + public static final String ORIGIN_MESSAGE_HEADER = "origin_message_header"; + + public static final String ORIGIN_SOURCE_NAME = "origin_offset_name"; + + public static final String SHUFFLE_KEY = "SHUFFLE_KEY"; + + public static final String ORIGIN_MESSAGE_TRACE_ID = "origin_request_id"; + + protected static final String SHUFFLE_QUEUE_ID = "SHUFFLE_QUEUE_ID"; + protected static final String SHUFFLE_MESSAGES = "SHUFFLE_MESSAGES"; + /** + * 消息所属的window + */ + protected transient String MSG_OWNER = "MSG_OWNER"; + + private static final String SHUFFLE_TRACE_ID = "SHUFFLE_TRACE_ID"; + + private transient ShuffleMQCreator shuffleMQCreator; + + + + protected String shuffleOwnerName;//shuffle 拥有者到名子,如是窗口,则是windowname+groupname+updateflag + protected String windowName;//Provide objects generated by shuffle key, such as window objects + protected int splitCount; + + + protected transient IWindow window;//generator shuffle key + /** + * used for shuffle + */ + protected transient AbstractSupportShuffleSink producer;//sink for shuffle + + protected transient ShuffleMsgCache shuffleMsgCache = new ShuffleMsgCache(); + + + protected transient boolean isWindowTest = false;//used for test, can fired quickly + protected transient AtomicLong COUNT = new AtomicLong(0);//usded for test + + /** + * Layer 2 cache,Each piece is divided into multiple pieces + */ + protected class ShuffleMsgCache extends AbstractMultiSplitMessageCache> { + + public ShuffleMsgCache() { + super(messages -> { + if (messages == null || messages.size() == 0) { + return true; + } + ISplit split = messages.get(0).getLeft(); + JSONObject jsonObject = messages.get(0).getRight(); + JSONArray allMsgs = jsonObject.getJSONArray(SHUFFLE_MESSAGES); + for (int i = 1; i < messages.size(); i++) { + Pair pair = messages.get(i); + JSONObject msg = pair.getRight(); + JSONArray jsonArray = msg.getJSONArray(SHUFFLE_MESSAGES); + if (jsonArray != null) { + allMsgs.addAll(jsonArray); + } + } + JSONObject zipJsonObject = new JSONObject(); + zipJsonObject.put(COMPRESSION_MSG_DATA, CompressUtil.gZip(jsonObject.toJSONString())); + zipJsonObject.put(IS_COMPRESSION_MSG, true); + producer.batchAdd(new Message(zipJsonObject), split); + producer.flush(split.getQueueId()); + + return true; + }); + } + + @Override + protected String createSplitId(Pair msg) { + return msg.getLeft().getQueueId(); + } + } + + @Override protected boolean initConfigurable() { + this.sink=new AbstractSink() { + @Override protected boolean batchInsert(List messages) { + addMsg2ShuffleCache(messages); + return true; + } + }; + this.sink.init(); + + + isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest"); + return super.initConfigurable(); + } + protected transient AtomicBoolean hasNotify=new AtomicBoolean(false); + @Override protected IMessage doSink(IMessage message, AbstractContext context) { + if(hasNotify.compareAndSet(false,true)){ + notifyShuffleConsumerStart(); + } + return super.doSink(message, context); + } + + /** + * When the producer receives the first piece of data, it will notify the consumer to start consumption + */ + protected void notifyShuffleConsumerStart() { + AbstractStage consumerStage=(AbstractStage)((ChainPipeline)pipeline).getStageMap().get(nextStageLabels.get(0)); + IMessage message=new Message(new JSONObject()); + consumerStage.doMessage(message,new Context(message)); + } + + /** + * + * @param messageList + * @return + */ + protected boolean addMsg2ShuffleCache(List messageList) { + if(window!=null){ + //call back window domessage + for(IMessage message:messageList){ + window.doMessage(message,new Context(message)); + } + } + Map shuffleMap = keyByMsg(messageList); + if (shuffleMap != null && shuffleMap.size() > 0) { + Set splitIds = new HashSet<>(); + + for (Map.Entry entry : shuffleMap.entrySet()) { + ISplit split = this.shuffleMQCreator.getQueueList().get(entry.getKey()); + JSONObject msg = createMsg(entry.getValue(), split); + + shuffleMsgCache.addCache(new MutablePair<>(split, msg)); + splitIds.add(split.getQueueId()); + + } + + } + if (isWindowTest) { + long count = COUNT.addAndGet(messageList.size()); + System.out.println(shuffleOwnerName + " send shuffle msg count is " + count); + shuffleMsgCache.flush(); + } + return true; + } + + + + /** + * 对接收的消息按照不同shuffle key进行分组 + * + * @param messages + * @return + */ + protected Map keyByMsg(List messages) { + Map shuffleMap = new HashMap<>(); + for (IMessage msg : messages) { + if (msg.getHeader().isSystemMessage()) { + continue; + } + + String shuffleKey = generateShuffleKey(msg); + if (StringUtil.isEmpty(shuffleKey)) { + shuffleKey = ""; + LOG.debug("there is no group by value in message! " + msg.getMessageBody().toString()); + //continue; + } + Integer index = this.shuffleMQCreator.hash(shuffleKey); + JSONObject body = msg.getMessageBody(); + String offset = msg.getHeader().getOffset(); + String queueId = msg.getHeader().getQueueId(); + + body.put(ORIGIN_OFFSET, offset); + body.put(ORIGIN_QUEUE_ID, queueId); + body.put(ORIGIN_QUEUE_IS_LONG, msg.getHeader().getMessageOffset().isLongOfMainOffset()); + body.put(ORIGIN_MESSAGE_HEADER, JSONObject.toJSONString(msg.getHeader())); + body.put(ORIGIN_MESSAGE_TRACE_ID, msg.getHeader().getTraceId()); + body.put(SHUFFLE_KEY, shuffleKey); + + + JSONArray jsonArray = shuffleMap.get(index); + if (jsonArray == null) { + jsonArray = new JSONArray(); + shuffleMap.put(index, jsonArray); + } + jsonArray.add(body); + + } + return shuffleMap; + } + + /** + * Combine multiple messages into one to reduce the pressure on the shuffle queue and improve throughput + * @param messages + * @param split + * @return + */ + protected JSONObject createMsg(JSONArray messages, ISplit split) { + JSONObject msg = new JSONObject(); + //分片id + msg.put(SHUFFLE_QUEUE_ID, split.getQueueId()); + //合并的消息 + msg.put(SHUFFLE_MESSAGES, messages); + //消息owner + msg.put(MSG_OWNER, shuffleOwnerName); + // + try { + List traceList = new ArrayList<>(); + List groupByList = new ArrayList<>(); + for (int i = 0; i < messages.size(); i++) { + JSONObject object = messages.getJSONObject(i); + groupByList.add(object.getString("SHUFFLE_KEY")); + traceList.add(object.getString(ORIGIN_MESSAGE_TRACE_ID)); + } + String traceInfo = StringUtils.join(traceList); + String groupInfo = StringUtils.join(groupByList); + msg.put(SHUFFLE_TRACE_ID, StringUtils.join(traceList)); + TraceUtil.debug(traceInfo, "origin message out", split.getQueueId(), groupInfo, getConfigureName()); + } catch (Exception e) { + //do nothing + } + return msg; + } + + + + /** + * If it is a window shuffle, calculate the shuffle key through the window object; otherwise, give the shuffle key randomly according to the number of queues + * + * @param message + * @return + */ + protected String generateShuffleKey(IMessage message){ + if(window!=null){ + return window.generateShuffleKey(message); + }else { + return (Math.random()*this.shuffleMQCreator.getQueueList().size()+1)+""; + } + } + + protected AbstractSupportShuffleSink createProducer(String name, ISource source) { + this.shuffleMQCreator=ShuffleMQCreator.createShuffleCreator(((ChainPipeline)this.getPipeline()).getSource(),getPipeline().getNameSpace(),getPipeline().getConfigureName(),shuffleOwnerName,splitCount); + return shuffleMQCreator.getProducer(); + } + + + + @Override public boolean isAsyncNode() { + return true; + } + + @Override + public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { + window=configurableService.queryConfigurable(windowName,IWindow.TYPE); + if(this.producer==null){ + this.producer=createProducer(shuffleOwnerName,((ChainPipeline)this.getPipeline()).getSource()); + } + } + @Override + protected boolean openMockChannel() { + return false; + } + + public String getShuffleOwnerName() { + return shuffleOwnerName; + } + + public void setShuffleOwnerName(String shuffleOwnerName) { + this.shuffleOwnerName = shuffleOwnerName; + } + + public String getWindowName() { + return windowName; + } + + public void setWindowName(String windowName) { + this.windowName = windowName; + } + + public int getSplitCount() { + return splitCount; + } + + public void setSplitCount(int splitCount) { + this.splitCount = splitCount; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/SubPiplineChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/SubPiplineChainStage.java deleted file mode 100644 index 7fc90a15..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/SubPiplineChainStage.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.streams.common.topology.stages; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; -import org.apache.rocketmq.streams.common.channel.source.systemmsg.NewSplitMessage; -import org.apache.rocketmq.streams.common.channel.source.systemmsg.RemoveSplitMessage; -import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage; -import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; -import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.topology.ChainStage; -import org.apache.rocketmq.streams.common.topology.model.IStageHandle; -import org.apache.rocketmq.streams.common.topology.model.Pipeline; -import org.apache.rocketmq.streams.common.topology.task.StreamsTask; - -public class -SubPiplineChainStage extends ChainStage implements IAfterConfigurableRefreshListener { - - private static final Log LOG = LogFactory.getLog(SubPiplineChainStage.class); - - @ENVDependence - protected String filterMsgFieldNames;//需要去重的字段列表,用逗号分割 - @ENVDependence - protected String filterMsgSwitch;//开启过滤的开关 - protected transient StreamsTask streamsTask; - - protected transient IStageHandle handle = new IStageHandle() { - @Override - protected IMessage doProcess(IMessage message, AbstractContext context) { - streamsTask.doMessage(message, context); - return null; - } - - @Override - public String getName() { - return SubPiplineChainStage.class.getName(); - } - }; - - @Override - public boolean isAsyncNode() { - for (Pipeline pipline : streamsTask.getPipelines()) { - if (pipline.isAsynNode() == true) { - return true; - } - } - return false; - } - - @Override - public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { - sendSystem(message, context, streamsTask.getPipelines()); - } - - @Override - public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) { - sendSystem(message, context, streamsTask.getPipelines()); - } - - @Override - public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) { - sendSystem(message, context, streamsTask.getPipelines()); - } - - @Override - public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { - sendSystem(message, context, streamsTask.getPipelines()); - } - - /** - * 每隔一段时间会重新刷新数据,如果有新增的pipline会加载起来,如果有删除的会去除掉 - * - * @param configurableService - */ - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - if (streamsTask == null) { - streamsTask = new StreamsTask(); - streamsTask.setNameSpace(getPipeline().getNameSpace()); - streamsTask.setConfigureName(getPipeline().getConfigureName()); - if (this.filterMsgSwitch != null && this.filterMsgSwitch.equals("true")) { - streamsTask.setLogFingerprint(this.filterMsgFieldNames); - } - - // ChainPipeline chainPipeline=(ChainPipeline)getPipeline(); - // streamsTask.setParallelTasks(chainPipeline.getSource().getMaxThread()-1); - streamsTask.init(); - } - streamsTask.doProcessAfterRefreshConfigurable(configurableService); - } - - /** - * 把消息投递给pipline的channel,让子pipline完成任务 注意:子pipline对消息的任何修改,都不反映到当前的pipline - * - * @param t - * @param context - * @return - */ - @Override - protected IStageHandle selectHandle(T t, AbstractContext context) { - return handle; - } - - public String getFilterMsgFieldNames() { - return filterMsgFieldNames; - } - - public void setFilterMsgFieldNames(String filterMsgFieldNames) { - this.filterMsgFieldNames = filterMsgFieldNames; - } - - public String getFilterMsgSwitch() { - return filterMsgSwitch; - } - - public void setFilterMsgSwitch(String filterMsgSwitch) { - this.filterMsgSwitch = filterMsgSwitch; - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionChainStage.java index b66cb041..944fea61 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionChainStage.java @@ -24,8 +24,9 @@ import org.apache.rocketmq.streams.common.topology.ChainPipeline; /** - * 会把消息复制后转给其他的pipeline 主要处理场景是类似blink多任务过滤部分抽取成规则放到一个任务的场景 + * 新的解析已经废弃,主要兼容老的规则数据 */ +@Deprecated public class UnionChainStage extends AbstractMutilPipelineChainPipline { private static final long serialVersionUID = -6448769339534974034L; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionEndChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionEndChainStage.java new file mode 100644 index 00000000..f9b47193 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionEndChainStage.java @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.common.topology.stages; + +public class UnionEndChainStage extends EmptyChainStage { +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionStartChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionStartChainStage.java new file mode 100644 index 00000000..ae934843 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/UnionStartChainStage.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.rocketmq.streams.common.topology.stages; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.Context; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; + +public class UnionStartChainStage extends EmptyChainStage { + + //每个pipline,对应一个消息来源,在消息头上会有消息来源的name,根据name转发数据 + protected Map> msgSource2StageLables; + + + @Override + protected IMessage proccessMessage(IMessage message, AbstractContext context) { + if (CollectionUtil.isEmpty(msgSource2StageLables)) { + return message; + } + String msgSourceName = message.getHeader().getMsgRouteFromLable(); + Set lableNames=msgSource2StageLables.get(msgSourceName); + if(lableNames!=null){ + for(String lableName:lableNames){ + message.getHeader().addRouteLabel(lableName); + } + } + + return message; + } + + public Map> getMsgSource2StageLables() { + return msgSource2StageLables; + } + + public void setMsgSource2StageLables(Map> msgSource2StageLables) { + this.msgSource2StageLables = msgSource2StageLables; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ViewChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ViewChainStage.java new file mode 100644 index 00000000..553036ac --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/ViewChainStage.java @@ -0,0 +1,529 @@ +/* + * 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.rocketmq.streams.common.topology.stages; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.ServiceLoader; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; +import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; +import org.apache.rocketmq.streams.common.channel.impl.view.ViewSink; +import org.apache.rocketmq.streams.common.channel.sink.ISink; +import org.apache.rocketmq.streams.common.channel.source.systemmsg.NewSplitMessage; +import org.apache.rocketmq.streams.common.channel.source.systemmsg.RemoveSplitMessage; +import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; +import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.Context; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.optimization.IHomologousOptimization; +import org.apache.rocketmq.streams.common.optimization.MessageGlobleTrace; +import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; +import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintMetric; +import org.apache.rocketmq.streams.common.schedule.ScheduleManager; +import org.apache.rocketmq.streams.common.schedule.ScheduleTask; +import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.model.IStageHandle; +import org.apache.rocketmq.streams.common.topology.model.Pipeline; +import org.apache.rocketmq.streams.common.topology.task.TaskAssigner; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; + +public class ViewChainStage extends OutputChainStage implements IAfterConfigurableRefreshListener { + private static final Log LOG = LogFactory.getLog(ViewChainStage.class); + + /** + * 动态加载的pipeline,pipeline的source type是view,且tablename=sink的view name + */ + protected transient List> pipelines = new ArrayList<>(); + /** + * 是否开启优化 + */ + protected transient Boolean isOpenOptimization = true; + + /** + * Used for fingerprint filtering + */ + @ENVDependence protected String logFingerprint; + @ENVDependence protected String logFingerprintSwitch; + /** + * Homologous expression result cache + */ + protected int homologousRulesCaseSize = 2000000; + protected int homologousExpressionCaseSize = 2000000; + + /** + * Pre fingerprint filtering + */ + protected int preFingerprintCaseSize = 2000000; + protected int parallelTasks = 4; + /** + * fingerprint cache + */ + protected transient FingerprintCache homologousRulesCache; + /** + * Automatically parses pipelines, generates pre-filter fingerprints and expression estimates + */ + protected transient IHomologousOptimization homologousOptimization; + + /** + * Supports mul-threaded task execution + */ + protected transient ExecutorService executorService; + /** + * 总处理数据数 + */ + private final transient AtomicLong COUNT = new AtomicLong(0); + /** + * 触发规则的个数 + */ + private final transient AtomicLong FIRE_RULE_COUNT = new AtomicLong(0); + /** + * 最早的处理时间 + */ + protected transient Long firstReceiveTime = null; + + + + /** + * 是否包含key by节点 + */ + protected transient boolean isContainsKeyBy = false; + + protected transient IStageHandle handle = new IStageHandle() { + @Override + protected IMessage doProcess(IMessage message, AbstractContext context) { + if (CollectionUtil.isEmpty(pipelines)) { + return null; + } + + if (homologousOptimization == null && isOpenOptimization) { + synchronized (this) { + if (homologousOptimization == null) { + String isOpenOptimizationStr = ComponentCreator.getProperties().getProperty("homologous.optimization.switch"); + boolean isOpenOptimization = true; + if (StringUtil.isNotEmpty(isOpenOptimizationStr)) { + isOpenOptimization = Boolean.parseBoolean(isOpenOptimizationStr); + } + isOpenOptimization = isOpenOptimization; + if (isOpenOptimization) { + Iterable iterable = ServiceLoader.load(IHomologousOptimization.class); + Iterator it = iterable.iterator(); + if (it.hasNext()) { + homologousOptimization = it.next(); + homologousOptimization.optimizate(pipelines, homologousExpressionCaseSize, preFingerprintCaseSize); + } + } + } + } + } + //Calculate QPS + double qps = calculateQPS(); + if (StringUtil.isEmpty(logFingerprint)) { + if (COUNT.get() % 10000 == 0) { + System.out.println(getConfigureName() + " qps is " + qps + "。the count is " + COUNT.get()); + } + } + + if (homologousOptimization != null) { + homologousOptimization.calculate(message, context); + } + + boolean onlyOne = pipelines.size() == 1; + //Judge whether to turn on fingerprint filtering. When it is turned on, filter through fingerprint first + String msgKey = getFilterKey(message); + BitSetCache.BitSet bitSet = getFilterValue(msgKey); + boolean isHitCache = true; + CountDownLatch countDownLatch = null; + if (bitSet == null && StringUtil.isNotEmpty(logFingerprint)&&StringUtil.isNotEmpty(logFingerprintSwitch)&&Boolean.valueOf(logFingerprintSwitch)) { + bitSet = new BitSetCache.BitSet(pipelines.size()); + isHitCache = false; + if (!isContainsKeyBy && pipelines.size() > 1 && parallelTasks > 1) { + countDownLatch = new CountDownLatch(pipelines.size()); + } + } + int index = 0; + for (ChainPipeline pipeline :pipelines) { + //If the fingerprint matches, filter it directly + if (isHitCache && bitSet != null && bitSet.get(index)) { + index++; + continue; + } + + IMessage copyMessage = message; + if (!onlyOne) { + copyMessage = message.deepCopy(); + } + Context newContext = new Context(copyMessage); + newContext.setHomologousResult(context.getHomologousResult()); + newContext.setQuickFilterResult(context.getQuickFilterResult()); + HomologousTask homologousTask = new HomologousTask(copyMessage, newContext, pipeline, bitSet, index, msgKey); + if (executorService != null && countDownLatch != null) { + homologousTask.setCountDownLatch(countDownLatch); + executorService.execute(homologousTask); + } else { + homologousTask.run(); + } + index++; + } + + if (countDownLatch != null) { + try { + countDownLatch.await(); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + if (!isHitCache) { + addNoFireMessage(msgKey, bitSet); + } + if (StringUtil.isNotEmpty(logFingerprint)) { + printQPSWithFingerprint(qps); + } + + return null; + } + + @Override + public String getName() { + return ViewChainStage.class.getName(); + } + }; + + @Override + public boolean isAsyncNode() { + for (Pipeline pipline : this.pipelines) { + if (pipline.isAsynNode() == true) { + return true; + } + } + return false; + } + + @Override + public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { + sendSystem(message, context, this.pipelines); + } + + @Override + public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) { + sendSystem(message, context, this.pipelines); + } + + @Override + public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) { + sendSystem(message, context, this.pipelines); + } + + @Override + public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage checkPointMessage) { + sendSystem(message, context, this.pipelines); + } + + /** + * 每隔一段时间会重新刷新数据,如果有新增的pipline会加载起来,如果有删除的会去除掉 + * + * @param configurableService + */ + @Override + public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { + List> newPipelines = loadSubPipelines(configurableService); + boolean isChanged = false; + if(newPipelines==null){ + return; + } + + + List> deletePipeline = new ArrayList<>(); + if (newPipelines.size() > 0) { + for (ChainPipeline pipeline : newPipelines) { + if (!this.pipelines.contains(pipeline)) { + isChanged = true; + ScheduleManager.getInstance().regist(new ScheduleTask(0,10,pipeline)); + } + } + for (ChainPipeline pipeline : this.pipelines) { + if (!newPipelines.contains(pipeline)) { + isChanged = true; + deletePipeline.add(pipeline); + } + } + } + + if (isChanged) { + this.pipelines = newPipelines; + this.homologousRulesCache = new FingerprintCache(homologousRulesCaseSize); + for (ChainPipeline pipeline : deletePipeline) { + pipeline.destroy(); + } + } + + if (this.parallelTasks > 0 && executorService == null) { + executorService = ThreadPoolFactory.createThreadPool(this.parallelTasks); + } + } + @Override + public void setSink(ISink channel) { + ViewSink viewSink=(ViewSink)channel; + this.sink = channel; + this.setNameSpace(channel.getNameSpace()); + this.setSinkName(channel.getConfigureName()); + this.setLabel(channel.getConfigureName()); + this.setConfigureName(viewSink.getViewTableName()); + } + /** + * 动态装配子pipeline + * + * @param configurableService configurableService + */ + protected List> loadSubPipelines(IConfigurableService configurableService) { + List taskAssigners = configurableService.queryConfigurableByType(TaskAssigner.TYPE); + if (taskAssigners == null) { + return null; + } + String taskName = getConfigureName(); + List> subPipelines=new ArrayList<>(); + for (TaskAssigner taskAssigner : taskAssigners) { + if (!taskName.equals(taskAssigner.getTaskName())) { + continue; + } + String pipelineName = taskAssigner.getPipelineName(); + if(pipelineName!=null){ + ChainPipeline pipeline = configurableService.queryConfigurable(Pipeline.TYPE, pipelineName); + if (pipeline != null) { + subPipelines.add(pipeline); + } + } + } + return subPipelines; + } + + + + + /** + * When fingerprint is enabled, print QPS, filter rate, cache condition and rule matching rate + * + * @param qps qps + */ + protected void printQPSWithFingerprint(double qps) { + FingerprintMetric fingerprintMetric = this.homologousRulesCache.getOrCreateMetric(getOrCreateFingerNameSpace()); + double rate = fingerprintMetric.getHitCacheRate(); + double fireRate = (double) FIRE_RULE_COUNT.get() / (double) COUNT.get(); + if (COUNT.get() % 1000 == 0) { + System.out.println("qps is " + qps + ",the count is " + COUNT.get() + " the cache hit rate " + rate + " the cache size is " + fingerprintMetric.getCacheSize() + "," + "the fire rule rate is " + fireRate); + } + } + + + /** + * 如果确定这个message,在某个pipeline不触发,则记录下来,下次直接跳过,不执行 + * + * @param msgKey msgKey + * @param bitSet bitSet + */ + protected void addNoFireMessage(String msgKey, BitSetCache.BitSet bitSet) { + + if (StringUtil.isEmpty(logFingerprint)) { + return; + } + if(StringUtil.isEmpty(logFingerprintSwitch)){ + return ; + } + + if(Boolean.valueOf(logFingerprintSwitch)==false){ + return ; + } + this.homologousRulesCache.addLogFingerprint(getOrCreateFingerNameSpace(), msgKey, bitSet); + } + + protected BitSetCache.BitSet getFilterValue(String msgKey) { + if (StringUtil.isEmpty(logFingerprint)) { + return null; + } + if(StringUtil.isEmpty(logFingerprintSwitch)){ + return null; + } + + if(Boolean.valueOf(logFingerprintSwitch)==false){ + return null; + } + return this.homologousRulesCache.getLogFingerprint(getOrCreateFingerNameSpace(), msgKey); + } + + protected String getFilterKey(IMessage message) { + + if (StringUtil.isEmpty(logFingerprint)) { + return null; + } + + if(StringUtil.isEmpty(logFingerprintSwitch)){ + return null; + } + + if(Boolean.valueOf(logFingerprintSwitch)==false){ + return null; + } + + return FingerprintCache.creatFingerpringKey(message, getOrCreateFingerNameSpace(), this.logFingerprint); + } + + + + protected String getOrCreateFingerNameSpace() { + return getConfigureName(); + } + + /** + * Print QPS in the scene without fingerprint on + */ + protected double calculateQPS() { + if (firstReceiveTime == null) { + synchronized (this) { + if (firstReceiveTime == null) { + firstReceiveTime = System.currentTimeMillis(); + } + } + } + long second = ((System.currentTimeMillis() - firstReceiveTime) / 1000); + if (second == 0) { + second = 1; + } + return (double) (COUNT.incrementAndGet() / second); + } + /** + * 把消息投递给pipline的channel,让子pipline完成任务 注意:子pipline对消息的任何修改,都不反映到当前的pipline + * + * @param t + * @param context + * @return + */ + @Override + protected IStageHandle selectHandle(T t, AbstractContext context) { + return handle; + } + + + + class HomologousTask implements Runnable { + protected IMessage message; + protected AbstractContext context; + protected ChainPipeline pipeline; + protected BitSetCache.BitSet bitSet; + protected int index; + protected String msgKey; + protected CountDownLatch countDownLatch; + + public HomologousTask(IMessage message, AbstractContext context, ChainPipeline pipeline, BitSetCache.BitSet bitSet, int index, String msgKey) { + this.message = message; + this.context = context; + this.pipeline = pipeline; + this.bitSet = bitSet; + this.index = index; + this.msgKey = msgKey; + } + + @Override public void run() { + try { + pipeline.doMessage(message, context); + Boolean isFinish = MessageGlobleTrace.existFinishBranch(message); + if (isFinish != null && !isFinish) { + if (bitSet != null) { + bitSet.set(index); + } + } + Boolean existFinishBranch = MessageGlobleTrace.existFinishBranch(message); + if (existFinishBranch != null && existFinishBranch) { + FIRE_RULE_COUNT.incrementAndGet(); + } + } catch (Exception e) { + e.printStackTrace(); + LOG.error("pipeline execute error " + pipeline.getConfigureName(), e); + } finally { + if (this.countDownLatch != null) { + this.countDownLatch.countDown(); + } + } + } + + public CountDownLatch getCountDownLatch() { + return countDownLatch; + } + + public void setCountDownLatch(CountDownLatch countDownLatch) { + this.countDownLatch = countDownLatch; + } + } + + public int getHomologousRulesCaseSize() { + return homologousRulesCaseSize; + } + + public void setHomologousRulesCaseSize(int homologousRulesCaseSize) { + this.homologousRulesCaseSize = homologousRulesCaseSize; + } + + public int getHomologousExpressionCaseSize() { + return homologousExpressionCaseSize; + } + + public void setHomologousExpressionCaseSize(int homologousExpressionCaseSize) { + this.homologousExpressionCaseSize = homologousExpressionCaseSize; + } + + public int getPreFingerprintCaseSize() { + return preFingerprintCaseSize; + } + + public void setPreFingerprintCaseSize(int preFingerprintCaseSize) { + this.preFingerprintCaseSize = preFingerprintCaseSize; + } + + public int getParallelTasks() { + return parallelTasks; + } + + public void setParallelTasks(int parallelTasks) { + this.parallelTasks = parallelTasks; + } + + public String getLogFingerprint() { + return logFingerprint; + } + + public void setLogFingerprint(String logFingerprint) { + this.logFingerprint = logFingerprint; + } + + public String getLogFingerprintSwitch() { + return logFingerprintSwitch; + } + + public void setLogFingerprintSwitch(String logFingerprintSwitch) { + this.logFingerprintSwitch = logFingerprintSwitch; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFChainStage.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFChainStage.java index d44c4a07..537baa19 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFChainStage.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/stages/udf/UDFChainStage.java @@ -21,7 +21,6 @@ import org.apache.rocketmq.streams.common.configurable.IConfigurableService; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; import org.apache.rocketmq.streams.common.topology.model.IStageHandle; import org.apache.rocketmq.streams.common.topology.stages.AbstractStatelessChainStage; import org.apache.rocketmq.streams.common.utils.Base64Utils; @@ -33,9 +32,9 @@ public class UDFChainStage extends AbstractStatelessChainStage implements IAfterConfigurableRefreshListener { protected String udfOperatorClassSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码 protected transient StageBuilder selfChainStage; - protected transient PreFingerprint preFingerprint=null; - public UDFChainStage() {} + public UDFChainStage() { + } public UDFChainStage(StageBuilder selfOperator) { this.selfChainStage = selfOperator; @@ -43,27 +42,24 @@ public UDFChainStage(StageBuilder selfOperator) { udfOperatorClassSerializeValue = Base64Utils.encode(bytes); } - @Override - public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { + @Override public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { selfChainStage.checkpoint(message, context, checkPointMessage); } - @Override - public boolean isAsyncNode() { + @Override public boolean isAsyncNode() { return false; } - @Override - protected IStageHandle selectHandle(IMessage message, AbstractContext context) { + @Override protected IStageHandle selectHandle(IMessage message, AbstractContext context) { return selfChainStage.selectHandle(message, context); } @Override public IMessage doMessage(IMessage message, AbstractContext context) { super.doMessage(message, context); - if(!context.isContinue()&&this.filterFieldNames!=null&&context.get("_logfinger")!=null){ + if (!context.isContinue() && this.filterFieldNames != null && context.get("_logfinger") != null) { preFingerprint.addLogFingerprintToSource(message); } - if(context.get("NEED_USE_FINGER_PRINT")!=null){ + if (context.get("NEED_USE_FINGER_PRINT") != null) { context.remove("NEED_USE_FINGER_PRINT"); } return message; @@ -77,15 +73,12 @@ public void setUdfOperatorClassSerializeValue(String udfOperatorClassSerializeVa this.udfOperatorClassSerializeValue = udfOperatorClassSerializeValue; } - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - if(udfOperatorClassSerializeValue!=null){ + @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { + if (udfOperatorClassSerializeValue != null) { byte[] bytes = Base64Utils.decode(udfOperatorClassSerializeValue); selfChainStage = InstantiationUtil.deserializeObject(bytes); } - preFingerprint=loadLogFinger(); + preFingerprint = loadLogFinger(); } - - } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/StreamsTask.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/StreamsTask.java index 2ea26cef..38a6e7aa 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/StreamsTask.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/StreamsTask.java @@ -16,479 +16,145 @@ */ package org.apache.rocketmq.streams.common.topology.task; +import com.google.common.collect.Lists; import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.ServiceLoader; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; -import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; import org.apache.rocketmq.streams.common.configurable.IConfigurableService; -import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; -import org.apache.rocketmq.streams.common.context.AbstractContext; -import org.apache.rocketmq.streams.common.context.Context; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; -import org.apache.rocketmq.streams.common.optimization.IHomologousOptimization; -import org.apache.rocketmq.streams.common.optimization.MessageGlobleTrace; -import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; -import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintMetric; -import org.apache.rocketmq.streams.common.threadpool.ThreadPoolFactory; import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.model.Pipeline; -import org.apache.rocketmq.streams.common.topology.model.PipelineSourceJoiner; -import org.apache.rocketmq.streams.common.utils.CollectionUtil; -import org.apache.rocketmq.streams.common.utils.StringUtil; /** * run one or multi pipeline's */ -public class StreamsTask extends BasedConfigurable implements IStreamOperator, IAfterConfigurableRefreshListener { +public class StreamsTask extends BasedConfigurable implements IAfterConfigurableRefreshListener { private static final Log LOG = LogFactory.getLog(StreamsTask.class); public static final String TYPE = "stream_task"; - @ENVDependence - protected String logFingerprint; - protected int homologousRulesCaseSize = 2000000; - protected int homologousExpressionCaseSize = 2000000; - protected int preFingerprintCaseSize = 2000000; - protected int parallelTasks = 4; /** - * The pipeline or subtask executed in this task - */ - - protected transient List pipelines = new ArrayList<>(); - protected List pipelineNames = new ArrayList<>(); - /** - * fingerprint cache - */ - protected transient FingerprintCache homologousRulesCache; - /** - * Automatically parses pipelines, generates pre-filter fingerprints and expression estimates - */ - protected transient IHomologousOptimization homologousOptimization; - - /** - * Supports multi-threaded task execution + * 任务的状态,目前有started,stopped俩种, 任务序列化保存在数据库 */ - protected transient ExecutorService executorService; + public static final String STATE_STARTED = "started"; + public static final String STATE_STOPPED = "stopped"; + protected String state = "stopped"; /** - * 总处理数据数 - */ - private final transient AtomicLong COUNT = new AtomicLong(0); - /** - * 触发规则的个数 + * 在当前进程中任务的状态 */ - private final transient AtomicLong FIRE_RULE_COUNT = new AtomicLong(0); + protected transient AtomicBoolean isStarted = new AtomicBoolean(false); /** - * 最早的处理时间 + * The pipeline or subtask executed in this task */ - protected transient Long firstReceiveTime = null; + protected transient List> pipelines = new ArrayList<>(); + protected List pipelineNames = new ArrayList<>(); - /** - * 是否包含key by节点 - */ - protected transient boolean isContainsKeyBy = false; + public StreamsTask() { + setType(TYPE); + } public void start() { - Map hasStart = new HashMap<>(); - while (true) { - try { - for (ChainPipeline pipeline : pipelines) { - if (hasStart.containsKey(pipeline.getConfigureName())) { - continue; - } else { - startPipeline(pipeline); - hasStart.put(pipeline.getConfigureName(), true); - } - } - Thread.sleep(1000); - } catch (InterruptedException e) { - e.printStackTrace(); + if (this.isStarted.compareAndSet(false, true)) { + for (ChainPipeline pipeline : pipelines) { + startPipeline(pipeline); } } } - protected transient Boolean isOpenOptimization = true; - - @Override - public AbstractContext doMessage(IMessage message, AbstractContext context) { - if (CollectionUtil.isEmpty(this.pipelines)) { - return context; - } - - if (this.homologousOptimization == null && isOpenOptimization) { - synchronized (this) { - if (this.homologousOptimization == null) { - String isOpenOptimizationStr = ComponentCreator.getProperties().getProperty("homologous.optimization.switch"); - boolean isOpenOptimization = true; - if (StringUtil.isNotEmpty(isOpenOptimizationStr)) { - isOpenOptimization = Boolean.valueOf(isOpenOptimizationStr); - } - this.isOpenOptimization = isOpenOptimization; - if (isOpenOptimization) { - Iterable iterable = ServiceLoader.load(IHomologousOptimization.class); - Iterator it = iterable.iterator(); - if (it.hasNext()) { - this.homologousOptimization = it.next(); - this.homologousOptimization.optimizate(this.pipelines, this.homologousExpressionCaseSize, this.preFingerprintCaseSize); - } - } - } - } - } - /** - * Calculate QPS - */ - double qps = calculateQPS(); - if (StringUtil.isEmpty(logFingerprint)) { - if (COUNT.get() % 10000 == 0) { - System.out.println(getConfigureName() + " qps is " + qps + "。the count is " + COUNT.get()); - } - } - - if (homologousOptimization != null) { - homologousOptimization.calculate(message, context); - } - - boolean onlyOne = this.pipelines.size() == 1; - /** - * Judge whether to turn on fingerprint filtering. When it is turned on, filter through fingerprint first - */ - String msgKey = getFilterKey(message); - BitSetCache.BitSet bitSet = getFilterValue(msgKey); - boolean isHitCache = true; - CountDownLatch countDownLatch = null; - if (bitSet == null && StringUtil.isNotEmpty(this.logFingerprint)) { - bitSet = new BitSetCache.BitSet(this.pipelines.size()); - isHitCache = false; - if (!isContainsKeyBy && this.pipelines.size() > 1) { - countDownLatch = new CountDownLatch(pipelines.size()); - } - } - int index = 0; - for (ChainPipeline pipeline : this.pipelines) { - //If the fingerprint matches, filter it directly - if (isHitCache && bitSet != null && bitSet.get(index)) { - index++; - continue; - } - - IMessage copyMessage = message; - if (!onlyOne) { - copyMessage = message.deepCopy(); - } - Context newContext = new Context(copyMessage); - newContext.setHomologousResult(context.getHomologousResult()); - newContext.setQuickFilterResult(context.getQuickFilterResult()); - HomologousTask homologousTask = new HomologousTask(copyMessage, newContext, pipeline, bitSet, index, msgKey); - if (this.executorService != null) { - if (countDownLatch != null) { - homologousTask.setCountDownLatch(countDownLatch); - } - executorService.execute(homologousTask); - } else { - homologousTask.run(); - } - index++; - } - - if (countDownLatch != null) { - try { - countDownLatch.await(); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } - if (!isHitCache) { - addNoFireMessage(msgKey, bitSet); + @Override public void destroy() { + for (ChainPipeline pipeline : pipelines) { + pipeline.destroy(); } - if (StringUtil.isNotEmpty(this.logFingerprint)) { - printQPSWithFingerprint(qps); - } - - return null; } - @Override - public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - List taskAssigners = configurableService.queryConfigurableByType(TaskAssigner.TYPE); - if (taskAssigners == null) { - return; - } + @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { + + List> newPipelines = new ArrayList<>(); boolean isChanged = false; - List newPipelines = new ArrayList<>(); - for (TaskAssigner taskAssigner : taskAssigners) { - if (!getConfigureName().equals(taskAssigner.getTaskName())) { - continue; - } - List pipelineNames = taskAssigner.getPipelineNames(); - if (pipelineNames != null) { - for (String pipelineName : pipelineNames) { - ChainPipeline pipeline = configurableService.queryConfigurable(Pipeline.TYPE, pipelineName); - if (pipeline != null) { - newPipelines.add(pipeline); - } + if (this.pipelineNames != null && !pipelineNames.isEmpty()) { + for (String pipelineName : this.pipelineNames) { + ChainPipeline pipeline = configurableService.queryConfigurable(Pipeline.TYPE, pipelineName); + if (pipeline != null) { + newPipelines.add(pipeline); } } - } - loadSubPipelines(newPipelines, configurableService); - List deletePipeline = new ArrayList<>(); + + List> deletePipeline = new ArrayList<>(); if (newPipelines.size() > 0) { - for (ChainPipeline pipeline : newPipelines) { + for (ChainPipeline pipeline : newPipelines) { if (!this.pipelines.contains(pipeline)) { isChanged = true; break; } } - for (ChainPipeline pipeline : this.pipelines) { + for (ChainPipeline pipeline : this.pipelines) { if (!newPipelines.contains(pipeline)) { isChanged = true; deletePipeline.add(pipeline); } } } + if (isChanged) { this.pipelines = newPipelines; - this.homologousRulesCache = new FingerprintCache(homologousRulesCaseSize); - for (ChainPipeline pipeline : deletePipeline) { + for (ChainPipeline pipeline : deletePipeline) { pipeline.destroy(); } } - if (this.parallelTasks > 0) { - executorService = ThreadPoolFactory.createThreadPool(this.parallelTasks); - } - } - - /** - * 动态装配子pipline - * - * @param pipelines - * @param configurableService - */ - protected void loadSubPipelines(List pipelines, IConfigurableService configurableService) { - List joiners = configurableService.queryConfigurableByType(PipelineSourceJoiner.TYPE); - if (joiners == null) { - return; - } - String pipelineName = getConfigureName(); - for (PipelineSourceJoiner joiner : joiners) { - if (pipelineName.equals(joiner.getSourcePipelineName())) { - ChainPipeline pipeline = configurableService.queryConfigurable(Pipeline.TYPE, joiner.getPipelineName()); - if (pipeline != null) { - pipelines.add(pipeline); - } - } - } - } - - /** - * 如果确定这个message,在某个pipeline不触发,则记录下来,下次直接跳过,不执行 - * - * @param msgKey - * @param bitSet - */ - protected void addNoFireMessage(String msgKey, BitSetCache.BitSet bitSet) { - - if (StringUtil.isEmpty(logFingerprint)) { - return; - } - this.homologousRulesCache.addLogFingerprint(getOrCreateFingerNameSpace(), msgKey, bitSet); - } - - protected BitSetCache.BitSet getFilterValue(String msgKey) { - if (StringUtil.isEmpty(logFingerprint)) { - return null; - } - - return this.homologousRulesCache.getLogFingerprint(getOrCreateFingerNameSpace(), msgKey); - } - - protected String getFilterKey(IMessage message) { - - if (StringUtil.isEmpty(logFingerprint)) { - return null; - } - - return FingerprintCache.creatFingerpringKey(message, getOrCreateFingerNameSpace(), this.logFingerprint); - } - - protected String getOrCreateFingerNameSpace() { - return getConfigureName(); - } - - /** - * Print QPS in the scene without fingerprint on - */ - protected double calculateQPS() { - if (firstReceiveTime == null) { - synchronized (this) { - if (firstReceiveTime == null) { - firstReceiveTime = System.currentTimeMillis(); - } - } - } - long second = ((System.currentTimeMillis() - firstReceiveTime) / 1000); - if (second == 0) { - second = 1; - } - return (double) (COUNT.incrementAndGet() / second); - } - - /** - * When fingerprint is enabled, print QPS, filter rate, cache condition and rule matching rate - * - * @param qps - */ - protected void printQPSWithFingerprint(double qps) { - FingerprintMetric fingerprintMetric = this.homologousRulesCache.getOrCreateMetric(getOrCreateFingerNameSpace()); - double rate = fingerprintMetric.getHitCacheRate(); - double fireRate = (double) FIRE_RULE_COUNT.get() / (double) COUNT.get(); - if (COUNT.get() % 1000 == 0) { - System.out.println( - "qps is " + qps + ",the count is " + COUNT.get() + " the cache hit rate " + rate - + " the cache size is " + fingerprintMetric.getCacheSize() + "," - + "the fire rule rate is " + fireRate); - } } /** * start one pipeline * - * @param pipeline + * @param pipeline pipeline */ - protected void startPipeline(ChainPipeline pipeline) { - Thread thread = new Thread(new Runnable() { - @Override - public void run() { - pipeline.startChannel(); - } - }); + protected void startPipeline(ChainPipeline pipeline) { + Thread thread = new Thread(pipeline::startChannel); thread.start(); } - class HomologousTask implements Runnable { - protected IMessage message; - protected AbstractContext context; - protected ChainPipeline pipeline; - protected BitSetCache.BitSet bitSet; - protected int index; - protected String msgKey; - protected CountDownLatch countDownLatch; - - public HomologousTask(IMessage message, AbstractContext context, ChainPipeline pipeline, - BitSetCache.BitSet bitSet, int index, String msgKey) { - this.message = message; - this.context = context; - this.pipeline = pipeline; - this.bitSet = bitSet; - this.index = index; - this.msgKey = msgKey; - } - - @Override - public void run() { - try { - pipeline.doMessage(message, context); - Boolean isFinish = MessageGlobleTrace.existFinishBranch(message); - if (isFinish != null && !isFinish) { - if (bitSet != null) { - bitSet.set(index); - } - } - Boolean existFinishBranch = MessageGlobleTrace.existFinishBranch(message); - if (existFinishBranch != null && existFinishBranch) { - FIRE_RULE_COUNT.incrementAndGet(); - } - } catch (Exception e) { - e.printStackTrace(); - LOG.error("pipeline execute error " + pipeline.getConfigureName(), e); - } finally { - if (this.countDownLatch != null) { - this.countDownLatch.countDown(); - } - } - } - - public CountDownLatch getCountDownLatch() { - return countDownLatch; - } - - public void setCountDownLatch(CountDownLatch countDownLatch) { - this.countDownLatch = countDownLatch; - } - } - - public int getParallelTasks() { - return parallelTasks; - } - - public void setParallelTasks(int parallelTasks) { - this.parallelTasks = parallelTasks; - } - - public List getPipelines() { + public List> getPipelines() { return pipelines; } - public void setPipelines(List pipelines) { + public void setPipelines(List> pipelines) { this.pipelines = pipelines; + List pipelineNames = Lists.newArrayList(); for (ChainPipeline pipeline : this.pipelines) { - this.pipelineNames.add(pipeline.getConfigureName()); + pipelineNames.add(pipeline.getConfigureName()); } + this.pipelineNames = pipelineNames; } - public String getLogFingerprint() { - return logFingerprint; - } - - public void setLogFingerprint(String logFingerprint) { - this.logFingerprint = logFingerprint; - } - - public int getHomologousRulesCaseSize() { - return homologousRulesCaseSize; - } - - public void setHomologousRulesCaseSize(int homologousRulesCaseSize) { - this.homologousRulesCaseSize = homologousRulesCaseSize; - } - - public int getHomologousExpressionCaseSize() { - return homologousExpressionCaseSize; + public List getPipelineNames() { + return pipelineNames; } - public void setHomologousExpressionCaseSize(int homologousExpressionCaseSize) { - this.homologousExpressionCaseSize = homologousExpressionCaseSize; + public void setPipelineNames(List pipelineNames) { + this.pipelineNames = pipelineNames; } - public int getPreFingerprintCaseSize() { - return preFingerprintCaseSize; + public AtomicBoolean getIsStarted() { + return isStarted; } - public void setPreFingerprintCaseSize(int preFingerprintCaseSize) { - this.preFingerprintCaseSize = preFingerprintCaseSize; + public void setIsStarted(AtomicBoolean isStarted) { + this.isStarted = isStarted; } - public List getPipelineNames() { - return pipelineNames; + public String getState() { + return state; } - public void setPipelineNames(List pipelineNames) { - this.pipelineNames = pipelineNames; + public void setState(String state) { + this.state = state; } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/TaskAssigner.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/TaskAssigner.java index ca047ef4..2026d3d5 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/TaskAssigner.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/task/TaskAssigner.java @@ -16,15 +16,17 @@ */ package org.apache.rocketmq.streams.common.topology.task; -import java.util.List; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +/** + * 帮助某个pipline 关联对应的数据源pipline。 主要用于多个同源任务自动装配的场景 namespace:source pipline的name space name:piplineName + */ public class TaskAssigner extends BasedConfigurable { - public static String TYPE="taskAssinger"; - protected String taskName; - protected List pipelineNames; + public static String TYPE = "assigner"; + protected String taskName;//数据源对应的pipline name + protected String pipelineName;//需要关联的pipline name - public TaskAssigner(){ + public TaskAssigner() { setType(TYPE); } @@ -36,11 +38,11 @@ public void setTaskName(String taskName) { this.taskName = taskName; } - public List getPipelineNames() { - return pipelineNames; + public String getPipelineName() { + return pipelineName; } - public void setPipelineNames(List pipelineNames) { - this.pipelineNames = pipelineNames; + public void setPipelineName(String pipelineName) { + this.pipelineName = pipelineName; } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ConfigurableUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ConfigurableUtil.java index 50b1c23e..c6f5786f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ConfigurableUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ConfigurableUtil.java @@ -86,9 +86,9 @@ public static boolean compare(IConfigurable configurable1, IConfigurable configu if (!configurable1.getConfigureName().equals(configurable2.getConfigureName())) { return false; } - if (AbstractConfigurable.class.isInstance(configurable1) && AbstractConfigurable.class.isInstance(configurable2)) { - AbstractConfigurable abstractConfigurable1 = (AbstractConfigurable) configurable1; - AbstractConfigurable abstractConfigurable2 = (AbstractConfigurable) configurable2; + if (BasedConfigurable.class.isInstance(configurable1) && BasedConfigurable.class.isInstance(configurable2)) { + BasedConfigurable abstractConfigurable1 = (BasedConfigurable)configurable1; + BasedConfigurable abstractConfigurable2 = (BasedConfigurable)configurable2; if (abstractConfigurable1.getUpdateFlag() == abstractConfigurable2.getUpdateFlag()) { return true; } else { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ContantsUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ContantsUtil.java index 7f0513ad..261515d8 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ContantsUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ContantsUtil.java @@ -21,12 +21,13 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.rocketmq.streams.common.model.NameCreator; public class ContantsUtil { private static final List CONSTANTS_SIGNS = new ArrayList<>();//对于特殊字符优先处理,里面存储需要特殊处理的字符 private static final Map CONSTANTS_SIGNS_REPLACE = new HashMap<>();// 特殊字符和替换字符的映射 private static final Map CONSTANTS_REPLACE_SIGNS = new HashMap<>();//替换字符和特殊字符的映射 - + private static final NameCreator EscapesNameCreator=new NameCreator(); static { CONSTANTS_SIGNS.add("\\\\"); CONSTANTS_SIGNS.add("\\\""); @@ -51,6 +52,23 @@ public class ContantsUtil { CONSTANTS_REPLACE_SIGNS.put("#######", "'"); } + public static String replaceEscape(String str,Map flag2Escapes){ + int index=str.indexOf("\\"); + if(index==-1){ + return str; + } + String word=str.substring(index,index+1); + String flag= EscapesNameCreator.createName("escapes"); + str=str.substring(0,index)+flag+str.substring(index+1); + return replaceEscape(str,flag2Escapes); + } + public static void main(String[] args) { + String str = "34432\"fs"; + Map replaceEscape = new HashMap<>(16); + String value = replaceEscape(str, replaceEscape); + System.out.println(value); + } + /** * 替换特殊字符为替换字符串 * @@ -328,18 +346,7 @@ protected static boolean isContantsEnd(String expressionStr, String sign, int i, return true; } - public static void main(String[] args) { - String str = "splitarray('da''fsfds''ta','fdsdfs')"; - Map flag2ExpressionStr = new HashMap<>(16); - String value = doConstantReplace(str, flag2ExpressionStr, 1); - String[] values = value.split(","); - int i = 0; - for (String v : values) { - values[i] = restore(v, flag2ExpressionStr); - System.out.println(values[i]); - } - System.out.println(value); - } + public static boolean containContant(String jsonValue) { if (isContant(jsonValue)) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java index c6e8c229..bdad069b 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java @@ -196,8 +196,7 @@ public static DataType getDataTypeFromClass(Class clazz) { * @return DataType */ public static DataType getDataType(String dataTypeName) { - DataType dataType = baseTypeDataTypeMap.get(dataTypeName); - return dataType; + return baseTypeDataTypeMap.get(dataTypeName); } /** @@ -366,7 +365,18 @@ public static DataType createFieldDataType(Object object, String fieldName) { public static DataType createFieldDataType(Class clazz, String fieldName) { Method method = ReflectUtil.getGetMethod(clazz, fieldName); - Type type = method.getGenericReturnType(); + Type type =null; + if (method == null) { + try { + Field field = clazz.getDeclaredField(fieldName); + type = field.getType(); + } catch (NoSuchFieldException e) { + throw new RuntimeException(clazz.getName() + "." + fieldName+" not exist get method, please create get/set method for the field"); + } + + }else { + type = method.getGenericReturnType(); + } String typeString = type.toString(); if (typeString.startsWith("class ")) { typeString = null; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/FileUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/FileUtil.java index dad47e7e..0c5b3cbe 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/FileUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/FileUtil.java @@ -21,13 +21,16 @@ import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; +import java.io.FileOutputStream; import java.io.FileWriter; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.io.OutputStream; import java.net.JarURLConnection; import java.net.MalformedURLException; import java.net.URL; +import java.net.URLConnection; import java.util.ArrayList; import java.util.Enumeration; import java.util.List; @@ -36,6 +39,12 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +//import org.apache.http.client.config.RequestConfig; +//import org.apache.http.client.methods.CloseableHttpResponse; +//import org.apache.http.client.methods.HttpGet; +//import org.apache.http.impl.client.CloseableHttpClient; +//import org.apache.http.impl.client.HttpClients; +//import org.apache.log4j.lf5.util.StreamUtils; import org.apache.rocketmq.streams.common.interfaces.ILineMessageProcessor; public class FileUtil { @@ -147,7 +156,7 @@ public static boolean inJar(String dir, Class clazz) { } public static List getFileFromResoure2Target(String targetDir, String dirName, Class clazz, - String fileNameRegex, boolean supportNesting) { + String fileNameRegex, boolean supportNesting) { URL url = clazz.getClassLoader().getResource(dirName); if (url == null) { return null; @@ -156,7 +165,7 @@ public static List getFileFromResoure2Target(String targetDir, String dirN } public static List getFileFromDir2Target(String targetDir, String dirName, String fileNameRegex, - boolean supportNesting) { + boolean supportNesting) { File dir = new File(dirName); File[] files = dir.listFiles(); if (files == null) { @@ -197,7 +206,7 @@ public static List getFileFromDir2Target(String targetDir, String dirName, * @return */ public static List getFileFromJar2Target(String targetDir, String dirName, Class clazz, String fileNameRegex, - boolean supportNesting) { + boolean supportNesting) { URL url = null; BufferedReader br = null; BufferedWriter bw = null; @@ -206,7 +215,7 @@ public static List getFileFromJar2Target(String targetDir, String dirName, if (url == null) { return null; } - JarURLConnection jarURLConnection = (JarURLConnection)url.openConnection(); + JarURLConnection jarURLConnection = (JarURLConnection) url.openConnection(); JarFile jarFile = jarURLConnection.getJarFile(); Enumeration entries = jarFile.entries(); List fileList = new ArrayList<>(); @@ -562,7 +571,6 @@ public static boolean write(File file, List rows, boolean isAppend) { return false; } - for (String row : rows) { bw.write(row + LINE_SIGN); } @@ -751,7 +759,7 @@ protected static InputStream getJarInputStream(String fileName) { URL url = getJarFileURL(fileName); JarURLConnection jarConnection = null; try { - jarConnection = (JarURLConnection)url + jarConnection = (JarURLConnection) url .openConnection(); InputStream in = jarConnection.getInputStream(); return in; @@ -949,8 +957,14 @@ public static File findFile(List fileList, String fileName) { } public static void main(String[] args) { - File file = new File("http:sss"); - System.out.println(file.getName()); + long begin = System.currentTimeMillis(); +// File file = new File("/Users/yd/Documents/tert.jar"); +// downloadFile("https://yundun-bigdata.oss-cn-qingdao.aliyuncs.com/download/dipper/linux64/1.0.0/rocketmq-stream-sql_20220225112207911.jar", +// file); + downloadNet("https://yundun-bigdata.oss-cn-qingdao.aliyuncs.com/download/dipper/linux64/1.0.0/rocketmq-stream-sql_20220225112207911.jar", + "/Users/yd/Documents/tert.jar"); + long end = System.currentTimeMillis(); + System.out.println("用时=====" + (end - begin)); } /** @@ -972,4 +986,63 @@ public static boolean delFile(File file) { } return file.delete(); } + +// public static void downloadFile(String url, File destFile) { +// if (!url.contains("http") && !url.contains("https")) { +// url = "http://" + url; +// } +// try (CloseableHttpClient httpclient = HttpClients.createDefault()) { +// HttpGet httpget = new HttpGet(url); +// httpget.setConfig(RequestConfig.custom() // +// .setConnectionRequestTimeout(3000) // +// .setConnectTimeout(3000) // +// .setSocketTimeout(3000) // +// .build()); +// try (CloseableHttpResponse response = httpclient.execute(httpget)) { +// org.apache.http.HttpEntity entity = response.getEntity(); +//// File desc = new File(dest_file+File.separator+fileName); +// if (!destFile.exists()) { +// destFile.createNewFile(); +// } +// File folder = destFile.getParentFile(); +// folder.mkdirs(); +// try (InputStream is = entity.getContent(); // +// OutputStream os = new FileOutputStream(destFile)) { +// StreamUtils.copy(is, os); +// } +// }catch(Exception e){ +// throw new Throwable("文件下载失败......", e); +// } finally { +// } +// } catch (Throwable e) { +// e.printStackTrace(); +// } +//// return dest_file+File.separator+fileName; +// } + + public static void downloadNet(String packageUrl, String destPath) { + // 下载网络文件 + int bytesum = 0; + int byteread = 0; + + try { + URL url = new URL(packageUrl); + URLConnection conn = url.openConnection(); + InputStream inStream = conn.getInputStream(); + FileOutputStream fs = new FileOutputStream(destPath); + + byte[] buffer = new byte[2048]; + int length; + while ((byteread = inStream.read(buffer)) != -1) { + bytesum += byteread; + System.out.println(bytesum); + fs.write(buffer, 0, byteread); + } + } catch (FileNotFoundException e) { + e.printStackTrace(); + } catch (IOException e) { + e.printStackTrace(); + } + } + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/InstantiationUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/InstantiationUtil.java index 43286904..18382304 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/InstantiationUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/InstantiationUtil.java @@ -24,6 +24,7 @@ import java.io.ObjectOutputStream; import java.io.ObjectStreamClass; import java.util.HashMap; +import org.nustaq.serialization.FSTConfiguration; /** * 因为匿名类,不能很好的序列化,目前用java的序列化框架,单写一个序列化方法实现 @@ -31,6 +32,7 @@ * @return */ public class InstantiationUtil { + private static FSTConfiguration conf = FSTConfiguration.createAndroidDefaultConfiguration(); /** * 因为匿名类,不能很好的序列化,目前用java的序列化框架,单写一个序列化方法实现 @@ -38,14 +40,15 @@ public class InstantiationUtil { * @return */ public static byte[] serializeObject(Object o) { - try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ObjectOutputStream oos = new ObjectOutputStream(baos)) { - oos.writeObject(o); - oos.flush(); - return baos.toByteArray(); - } catch (IOException e) { - throw new RuntimeException("serializeAnonymousObject error ", e); - } + return conf.asByteArray(o); +// try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); +// ObjectOutputStream oos = new ObjectOutputStream(baos)) { +// oos.writeObject(o); +// oos.flush(); +// return baos.toByteArray(); +// } catch (IOException e) { +// throw new RuntimeException("serializeAnonymousObject error ", e); +// } } /** @@ -58,20 +61,21 @@ public static byte[] serializeObject(Object o) { * @throws ClassNotFoundException */ public static T deserializeObject(byte[] bytes) { - - final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - try { - ByteArrayInputStream in = new ByteArrayInputStream(bytes); - ObjectInputStream oois = new ClassLoaderObjectInputStream(in, classLoader); - Thread.currentThread().setContextClassLoader(classLoader); - return (T)oois.readObject(); - } catch (IOException e) { - throw new RuntimeException("serializeAnonymousObject error ", e); - } catch (ClassNotFoundException e) { - throw new RuntimeException("serializeAnonymousObject error ", e); - } finally { - Thread.currentThread().setContextClassLoader(classLoader); - } + return (T)conf.asObject(bytes); +// +// final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); +// try { +// ByteArrayInputStream in = new ByteArrayInputStream(bytes); +// ObjectInputStream oois = new ClassLoaderObjectInputStream(in, classLoader); +// Thread.currentThread().setContextClassLoader(classLoader); +// return (T)oois.readObject(); +// } catch (IOException e) { +// throw new RuntimeException("serializeAnonymousObject error ", e); +// } catch (ClassNotFoundException e) { +// throw new RuntimeException("serializeAnonymousObject error ", e); +// } finally { +// Thread.currentThread().setContextClassLoader(classLoader); +// } } public static class ClassLoaderObjectInputStream extends ObjectInputStream { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JsonableUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JsonableUtil.java index cb63e1f0..90856981 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JsonableUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/JsonableUtil.java @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.streams.common.utils; +import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; import com.google.gson.Gson; import com.google.gson.GsonBuilder; @@ -57,4 +58,8 @@ public static String formatJson(JSONObject jsonObject) { String value = gson.toJson(jsonObject); return value; } + public static String formatJson(JSONArray jsonObject) { + String value = gson.toJson(jsonObject); + return value; + } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/KryoUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/KryoUtil.java new file mode 100644 index 00000000..d6fa1072 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/KryoUtil.java @@ -0,0 +1,210 @@ +/* + * 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.rocketmq.streams.common.utils; + + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.commons.codec.binary.Base64; +import org.objenesis.strategy.StdInstantiatorStrategy; + +import java.io.*; + +/** + * Kryo Utils + *

+ */ +public class KryoUtil { + + private static final String DEFAULT_ENCODING = "UTF-8"; + + //每个线程的 Kryo 实例 + private static final ThreadLocal kryoLocal = new ThreadLocal() { + @Override + protected Kryo initialValue() { + Kryo kryo = new Kryo(); + + /** + * 不要轻易改变这里的配置!更改之后,序列化的格式就会发生变化, + * 上线的同时就必须清除 Redis 里的所有缓存, + * 否则那些缓存再回来反序列化的时候,就会报错 + */ + //支持对象循环引用(否则会栈溢出) + kryo.setReferences(true); //默认值就是 true,添加此行的目的是为了提醒维护者,不要改变这个配置 + + //不强制要求注册类(注册行为无法保证多个 JVM 内同一个类的注册编号相同;而且业务系统中大量的 Class 也难以一一注册) + kryo.setRegistrationRequired(false); //默认值就是 false,添加此行的目的是为了提醒维护者,不要改变这个配置 + + return kryo; + } + }; + + /** + * 获得当前线程的 Kryo 实例 + * + * @return 当前线程的 Kryo 实例 + */ + public static Kryo getInstance() { + return kryoLocal.get(); + } + + //----------------------------------------------- + // 序列化/反序列化对象,及类型信息 + // 序列化的结果里,包含类型的信息 + // 反序列化时不再需要提供类型 + //----------------------------------------------- + + /** + * 将对象【及类型】序列化为字节数组 + * + * @param obj 任意对象 + * @param 对象的类型 + * @return 序列化后的字节数组 + */ + public static byte[] writeToByteArray(T obj) { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + Output output = new Output(byteArrayOutputStream); + + Kryo kryo = getInstance(); + kryo.writeClassAndObject(output, obj); + output.flush(); + + return byteArrayOutputStream.toByteArray(); + } + + /** + * 将对象【及类型】序列化为 String + * 利用了 Base64 编码 + * + * @param obj 任意对象 + * @param 对象的类型 + * @return 序列化后的字符串 + */ + public static String writeToString(T obj) { + try { + return new String(Base64.encodeBase64(writeToByteArray(obj)), DEFAULT_ENCODING); + } catch (UnsupportedEncodingException e) { + throw new IllegalStateException(e); + } + } + + /** + * 将字节数组反序列化为原对象 + * + * @param byteArray writeToByteArray 方法序列化后的字节数组 + * @param 原对象的类型 + * @return 原对象 + */ + @SuppressWarnings("unchecked") + public static T readFromByteArray(byte[] byteArray) { + ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(byteArray); + Input input = new Input(byteArrayInputStream); + + Kryo kryo = getInstance(); + return (T) kryo.readClassAndObject(input); + } + + /** + * 将 String 反序列化为原对象 + * 利用了 Base64 编码 + * + * @param str writeToString 方法序列化后的字符串 + * @param 原对象的类型 + * @return 原对象 + */ + public static T readFromString(String str) { + try { + return readFromByteArray(Base64.decodeBase64(str.getBytes(DEFAULT_ENCODING))); + } catch (UnsupportedEncodingException e) { + throw new IllegalStateException(e); + } + } + + //----------------------------------------------- + // 只序列化/反序列化对象 + // 序列化的结果里,不包含类型的信息 + //----------------------------------------------- + + /** + * 将对象序列化为字节数组 + * + * @param obj 任意对象 + * @param 对象的类型 + * @return 序列化后的字节数组 + */ + public static byte[] writeObjectToByteArray(T obj) { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + Output output = new Output(byteArrayOutputStream); + + Kryo kryo = getInstance(); + kryo.writeObject(output, obj); + output.flush(); + + return byteArrayOutputStream.toByteArray(); + } + + /** + * 将对象序列化为 String + * 利用了 Base64 编码 + * + * @param obj 任意对象 + * @param 对象的类型 + * @return 序列化后的字符串 + */ + public static String writeObjectToString(T obj) { + try { + return new String(Base64.encodeBase64(writeObjectToByteArray(obj)), DEFAULT_ENCODING); + } catch (UnsupportedEncodingException e) { + throw new IllegalStateException(e); + } + } + + /** + * 将字节数组反序列化为原对象 + * + * @param byteArray writeToByteArray 方法序列化后的字节数组 + * @param clazz 原对象的 Class + * @param 原对象的类型 + * @return 原对象 + */ + @SuppressWarnings("unchecked") + public static T readObjectFromByteArray(byte[] byteArray, Class clazz) { + ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(byteArray); + Input input = new Input(byteArrayInputStream); + + Kryo kryo = getInstance(); + return kryo.readObject(input, clazz); + } + + /** + * 将 String 反序列化为原对象 + * 利用了 Base64 编码 + * + * @param str writeToString 方法序列化后的字符串 + * @param clazz 原对象的 Class + * @param 原对象的类型 + * @return 原对象 + */ + public static T readObjectFromString(String str, Class clazz) { + try { + return readObjectFromByteArray(Base64.decodeBase64(str.getBytes(DEFAULT_ENCODING)), clazz); + } catch (UnsupportedEncodingException e) { + throw new IllegalStateException(e); + } + } +} \ No newline at end of file diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/NameCreatorUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/NameCreatorUtil.java deleted file mode 100644 index 5b0f94dc..00000000 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/NameCreatorUtil.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.streams.common.utils; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; - -public class NameCreatorUtil { - - private transient AtomicInteger nameCreator = new AtomicInteger(10000); - - private static Map creatorMap = new HashMap<>(); - - /** - * 每个规则一个名字生成器,expression name - * - * @param names - * @return - */ - public static NameCreatorUtil createOrGet(String... names) { - String ruleName = MapKeyUtil.createKeyBySign("_", names); - NameCreatorUtil nameCreator = creatorMap.get(ruleName); - if (nameCreator != null) { - return nameCreator; - } - synchronized (NameCreatorUtil.class) { - nameCreator = creatorMap.get(ruleName); - if (nameCreator != null) { - return nameCreator; - } - nameCreator = new NameCreatorUtil(); - creatorMap.put(ruleName, nameCreator); - } - return nameCreator; - } - - public static String createNewName(String... names) { - NameCreatorUtil nameCreator = createOrGet(names); - return nameCreator.createName(names); - } - - protected String createName(String... namePrefix) { - return MapKeyUtil.createKeyBySign("_", MapKeyUtil.createKeyBySign("_", namePrefix), nameCreator.incrementAndGet() + ""); - } -} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PipelineHTMLUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PipelineHTMLUtil.java new file mode 100644 index 00000000..8ae00828 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PipelineHTMLUtil.java @@ -0,0 +1,299 @@ +/* + * 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.rocketmq.streams.common.utils; + +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.JSONObject; +import java.util.List; +import java.util.logging.Filter; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; +import org.apache.rocketmq.streams.common.topology.metric.StageGroup; +import org.apache.rocketmq.streams.common.topology.model.AbstractRule; +import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; +import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; + +public class PipelineHTMLUtil { + public static String createHTML(ChainPipeline pipeline){ + StringBuilder html=new StringBuilder("\n" + + "\t\n" + + "\t\t\n" + + "\t\tTest\n" + + "\t\t\n" + + "\t\t\n" + + "\t\n" + + "\t"); + + + List stageGroups= pipeline.getRootStageGroups(); + int i=100; + StringBuilder hideHtmlBuilder=new StringBuilder(); + String baseTab="\t\t"; + html.append(baseTab+"

    \n"); + html.append(baseTab+"\tCREATE TABLE "+pipeline.getMsgSourceName()+"\n"); + + html.append(baseTab+"\t\n"); + hideHtmlBuilder.append(baseTab+"\t
  1. \n"); + hideHtmlBuilder.append(baseTab+"\t\t"+pipeline.getCreateTableSQL()+"\n"); + hideHtmlBuilder.append(baseTab+"\t
  2. \n"); + i++; + + html.append(baseTab+"\t\n"); + hideHtmlBuilder.append(baseTab+"\t
  3. \n"); + JSONObject jsonObject=JSON.parseObject(pipeline.getSource().toJson()); + jsonObject.remove("metaData"); + hideHtmlBuilder.append(baseTab+"\t\t"+JsonableUtil.formatJson(jsonObject)+"\n"); + hideHtmlBuilder.append(baseTab+"\t
  4. \n"); + + html.append(hideHtmlBuilder.toString()); + html.append(baseTab+"\n
\n"); + i=1000; + for(StageGroup stageGroup:stageGroups){ + String stageGroupHtml=createStageGroupHTML(stageGroup,i,2); + html.append(stageGroupHtml); + i=i+1000; + } + html.append("\t\n"); + html.append("\n"); + return html.toString(); + } + + private static String createStageGroupHTML(StageGroup group,int index,int tabSize) { + StringBuilder stringBuilder=new StringBuilder(); + StringBuilder hideHtmlBuilder=new StringBuilder(); + String baseTab=""; + for(int i=0;i\n"); + stringBuilder.append(baseTab+"\t"+group.getViewName()+"\n"); + + + stringBuilder.append(baseTab+"\t\n"); + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + hideHtmlBuilder.append(baseTab+"\t\t"+group.getSql()+"\n"); + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + index++; + + + if(group.getChildren().size()>0){ + stringBuilder.append(baseTab+"\t\n"); + + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + int i=0; + for(StageGroup subStageGroup:group.getChildren()){ + subStageGroup.calculateMetric(); + int subIndex=index*1000+(i*100); + hideHtmlBuilder.append(createStageGroupHTML(subStageGroup,subIndex,tabSize+2)); + i++; + } + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + index++; + } + + + + + if(group.getAllStages().size()>0){ + stringBuilder.append(baseTab+"\t\n"); + + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + int i=0; + for(AbstractStage stage:group.getAllStages()){ + int subIndex=index*1000+(i*100); + hideHtmlBuilder.append(createStageHTML(stage,subIndex,tabSize+2)); + i++; + } + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + index++; + + } + + + stringBuilder.append(baseTab+"\t\n"); + index++; + + + stringBuilder.append(baseTab+"\t\n"); + index++; + + + stringBuilder.append(baseTab+"\t\n"); + index++; + + index++; + + + + + stringBuilder.append(hideHtmlBuilder.toString()); + stringBuilder.append(baseTab+"\n\n"); + return stringBuilder.toString(); + } + + protected static String createStageHTML(AbstractStage stage, int index, int tabSize) { + StringBuilder stringBuilder=new StringBuilder(); + StringBuilder hideHtmlBuilder=new StringBuilder(); + String baseTab=""; + for(int i=0;i\n"); + stringBuilder.append(baseTab+"\t"+ stage.getClass().getSimpleName().replace("ChainStage","")+":"+stage.getLabel()+"\n"); + + + stringBuilder.append(baseTab+"\t\n"); + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + hideHtmlBuilder.append(baseTab+"\t\t"+stage.getSql()+"\n\n"); + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + index++; + + if(ScriptChainStage.class.isInstance(stage)){ + ScriptChainStage scriptChainStage=(ScriptChainStage)stage; + stringBuilder.append(baseTab+"\t\n"); + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + hideHtmlBuilder.append(baseTab+"\t\t"+scriptChainStage.getScript().getValue()+"\n\n"); + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + index++; + } + + + if(FilterChainStage.class.isInstance(stage)){ + FilterChainStage filterChainStage=(FilterChainStage)stage; + if(filterChainStage.getRules()!=null&&filterChainStage.getRules().size()>0){ + AbstractRule rule=(AbstractRule) filterChainStage.getRules().get(0); + stringBuilder.append(baseTab+"\t\n"); + + hideHtmlBuilder.append(baseTab+"\t\t
  • \n"); + hideHtmlBuilder.append(baseTab+"\t\t\t"+rule.toString()+"\n\n"); + hideHtmlBuilder.append(baseTab+"\t\t
  • \n"); + index++; + + if(stage.getStageMetric().getInCount()>stage.getStageMetric().getOutCount()){ + stringBuilder.append(baseTab+"\t\n"); + + + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + hideHtmlBuilder.append(baseTab+"\t\t"+filterChainStage.getStageMetric().createNotFireReason()+"\n\n"); + hideHtmlBuilder.append(baseTab+"\t
  • \n"); + + + index++; + } + + } + + } + + + stringBuilder.append(baseTab+"\t\n"); + index++; + + + + + + + + + stringBuilder.append(baseTab+"\t\n"); + index++; + + + + stringBuilder.append(baseTab+"\t\n"); + index++; + + stringBuilder.append(baseTab+"\t\n"); + index++; + + + stringBuilder.append(baseTab+"\t\n"); + index++; + + + stringBuilder.append(baseTab+"\t\n"); + index++; + + + stringBuilder.append(hideHtmlBuilder.toString()); + stringBuilder.append(baseTab+"\n\n"); + return stringBuilder.toString(); + + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PropertiesUtils.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PropertiesUtils.java index b07d61b9..50707b59 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PropertiesUtils.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/PropertiesUtils.java @@ -65,7 +65,7 @@ public static void putProperty(String line, String sign, Map properties) { public static Properties getResourceProperties(Class clazz, String propertiesPath) { URL url = clazz.getClassLoader().getResource(propertiesPath); if (url == null) { - LOG.error("can not load component's properties file " + propertiesPath); + // LOG.error("can not load component's properties file " + propertiesPath); return null; } BufferedReader br = null; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ReflectUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ReflectUtil.java index 77f1612c..702dca0d 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ReflectUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ReflectUtil.java @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.streams.common.utils; +import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; import java.lang.reflect.Constructor; @@ -167,7 +168,8 @@ public static Object deserializeObject(JSONObject jsonObject) { Object object = ReflectUtil.forInstance(className); scanFields(object, (o, field) -> { String fileName = field.getName(); - DataType dataType = DataTypeUtil.getDataTypeFromClass(field.getType()); + Class fieldClass=ReflectUtil.getBeanFieldType(o.getClass(),fileName); + DataType dataType = DataTypeUtil.getDataTypeFromClass(fieldClass); String genericTypeStr = null; if (field.getGenericType() != null) { genericTypeStr = field.getGenericType().toString(); @@ -194,10 +196,11 @@ public static Object deserializeObject(JSONObject jsonObject) { Object value = dataType.getData(fieldJson); try { if (value != null) { - field.setAccessible(true); - field.set(object, value); + ReflectUtil.setBeanFieldValue(object,field.getName(),value); +// field.setAccessible(true); +// field.set(object, value); } - } catch (IllegalAccessException e) { + } catch (Exception e) { throw new RuntimeException("Deserialize error ,the field " + fileName + " Deserialize error "); } }); @@ -220,7 +223,8 @@ public static JSONObject serializeObject(Object o) { scanFields(o, (o1, field) -> { field.setAccessible(true); String fileName = field.getName(); - DataType dataType = DataTypeUtil.getDataTypeFromClass(field.getType()); + Class fieldClass=ReflectUtil.getBeanFieldType(o1.getClass(),fileName); + DataType dataType = DataTypeUtil.getDataTypeFromClass(fieldClass); String genericTypeStr = null; if (field.getGenericType() != null) { genericTypeStr = field.getGenericType().toString(); @@ -241,12 +245,12 @@ public static JSONObject serializeObject(Object o) { } setDataTypeParadigmType(dataType, genericTypeStr, ParameterizedType.class.isInstance(field.getGenericType())); try { - Object value = field.get(o1); + Object value = ReflectUtil.getDeclaredField(o1,fileName); if (value == null) { return; } objectJson.put(fileName, dataType.toDataJson(value)); - } catch (IllegalAccessException e) { + } catch (Exception e) { throw new RuntimeException("serializeObject error ,the field " + fileName + " serialize error "); } @@ -286,8 +290,8 @@ public static void deserializeObject(Object object, byte[] bytes) { @Override public void doProcess(Object o, Field field) { field.setAccessible(true); - Class clazz = field.getType(); - DataType dataType = DataTypeUtil.getDataTypeFromClass(clazz); + Class fieldClass=ReflectUtil.getBeanFieldType(o.getClass(),field.getName()); + DataType dataType = DataTypeUtil.getDataTypeFromClass(fieldClass); String genericTypeStr = null; if (field.getGenericType() != null) { genericTypeStr = field.getGenericType().toString(); @@ -310,8 +314,8 @@ public void doProcess(Object o, Field field) { Object value = dataType.byteToValue(bytes, offset.get()); offset.addAndGet(dataType.toBytes(value, false).length); try { - field.set(object, value); - } catch (IllegalAccessException e) { + ReflectUtil.setBeanFieldValue(object,field.getName(),value); + } catch (Exception e) { throw new RuntimeException("can not set field value " + field.getName(), e); } } @@ -430,18 +434,24 @@ public static void scanFields(Object o, Class clazz, IFieldProcessor fieldProces if (fields == null) { return; } + for (Field field : fields) { - if (field.isAnnotationPresent(NoSerialized.class)) { - continue; - } - if (Modifier.isStatic(field.getModifiers())) { - continue; - } else if (Modifier.isTransient(field.getModifiers())) { - continue; - } else if (Modifier.isNative(field.getModifiers())) { - continue; + try { + if (field.isAnnotationPresent(NoSerialized.class)) { + continue; + } + if (Modifier.isStatic(field.getModifiers())) { + continue; + } else if (Modifier.isTransient(field.getModifiers())) { + continue; + } else if (Modifier.isNative(field.getModifiers())) { + continue; + } + fieldProcessor.doProcess(o, field); + }catch (Exception e){ + throw new RuntimeException("Error serializing object, class is "+clazz.getName()+", field is "+field.getName(),e); } - fieldProcessor.doProcess(o, field); + } Class parent = clazz.getSuperclass(); scanFields(o, parent, fieldProcessor, basedClass); @@ -546,7 +556,15 @@ public static T getBeanFieldOrJsonValue(Object bean, String fieldName) { result = getJsonItemValue(modelBean, name); } else if (Map.class.isInstance(modelBean)) { result = ((Map) modelBean).get(name); - } else { + } else if(String.class.isInstance(modelBean)){ + String value=(String)modelBean; + if (value.startsWith("[") && value.endsWith("]")) { + modelBean = JSON.parseArray(value); + } else { + modelBean = JSON.parseObject(value); + } + result=getJsonItemValue(modelBean,name); + }else { result = getDeclaredField(modelBean, name);// ChannelMessage.messageBody } modelBean = result; @@ -679,6 +697,7 @@ public static void setBeanFieldValue(Object object, String modelFieldName, Objec DataType dataType = DataTypeUtil.getDataTypeFromClass(fieldValue.getClass()); Object convertFieldValue = dataType.convert(fieldValue); if (method != null) { + method.setAccessible(true); method.invoke(object, convertFieldValue); } else { Field field = object.getClass().getDeclaredField(modelFieldName); @@ -753,6 +772,7 @@ private static T getFieldValue(Object bean, String fieldName) { if (method == null) { throw new RuntimeException("can not get " + fieldName + "'s value, the method is not exist"); } + method.setAccessible(true); return (T) method.invoke(bean); } catch (Exception e) { throw new RuntimeException("can not get " + fieldName + "'s value", e); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java index c7d21b30..60691cc6 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java @@ -32,11 +32,11 @@ import org.apache.rocketmq.streams.common.datatype.ArrayDataType; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.datatype.StringDataType; +import org.apache.rocketmq.streams.common.interfaces.ISerialize; public class SerializeUtil { private static final Log LOG = LogFactory.getLog(SerializeUtil.class); - - /** + /** * 把一个对象序列化成字节,对象中的字段是datatype支持的 * * @param object @@ -79,7 +79,18 @@ public static T deserialize(byte[] bytes, AtomicInteger offset) { DataType dataType = DataTypeUtil.getDataType(dataTypeName); return (T) dataType.byteToValue(bytes, offset); } - + /** + * 把一个对象的字段,通过字节填充,字段不能有null值 + * + * @param bytes + */ + public static T deserialize(byte[] bytes,Class clazz) { + if(ISerialize.class.isAssignableFrom(clazz)){ + return (T)KryoUtil.readObjectFromByteArray(bytes,clazz); + //return (T)conf.asObject(bytes); + } + return deserialize(bytes, new AtomicInteger(0)); + } /** * 把一个对象的字段,通过字节填充,字段不能有null值 * diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ServiceLoadUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ServiceLoadUtil.java new file mode 100644 index 00000000..77696f86 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/ServiceLoadUtil.java @@ -0,0 +1,63 @@ +/* + * 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.rocketmq.streams.common.utils; + +import java.util.ArrayList; +import java.util.List; +import java.util.ServiceLoader; +import org.apache.rocketmq.streams.common.cache.softreference.ICache; +import org.apache.rocketmq.streams.common.cache.softreference.impl.SoftReferenceCache; +import org.apache.rocketmq.streams.common.model.ServiceName; + +public class ServiceLoadUtil { + + private static ICache serviceCache=new SoftReferenceCache<>(); + public static T loadService(Class interfaceClass,String serviceName){ + List allService = (List)serviceCache.get(interfaceClass.getName()); + if(allService==null){ + allService=new ArrayList<>(); + Iterable iterable = ServiceLoader.load(interfaceClass); + for (T t : iterable) { + allService.add(t); + } + serviceCache.put(interfaceClass.getName(), allService); + } + if(CollectionUtil.isEmpty(allService)){ + return null; + } + if(StringUtil.isEmpty(serviceName)){ + return allService.get(0); + } + for(T t:allService){ + ServiceName annotation = (ServiceName)t.getClass().getAnnotation(ServiceName.class); + if (annotation == null) { + return null; + } + if (serviceName.equals(annotation.value())) { + return t; + } + if (serviceName.equals(annotation.aliasName())) { + return t; + } + + if (serviceName.equals(annotation.name())) { + return t; + } + } + return null; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/TraceUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/TraceUtil.java index b5762d7f..ac7b2024 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/TraceUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/TraceUtil.java @@ -93,17 +93,18 @@ private static boolean skip(String traceId) { * @return */ public static boolean hit(String traceId) { - String type = ComponentCreator.getProperties().getProperty("dipper.trace.service.switch"); - if (!RUNNING_MODE.equals(type)) { - return false; - } - for (String white : whiteSet) { - if (traceId.startsWith(white)) { - return true; - } - } - - return false; +// String type = ComponentCreator.getProperties().getProperty("dipper.trace.service.switch"); +// if (!RUNNING_MODE.equals(type)) { +// return false; +// } +// for (String white : whiteSet) { +// if (traceId.startsWith(white)) { +// return true; +// } +// } +// +// return false; + return true; } private static String envelope(String traceId, String[] messages) { diff --git a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/channel/SinkTest.java b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/channel/SinkTest.java index 89e42007..154c05dc 100644 --- a/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/channel/SinkTest.java +++ b/rocketmq-streams-commons/src/test/java/org/apache/rocketmq/streams/common/channel/SinkTest.java @@ -36,7 +36,7 @@ public void testSink() { msg.put("name", "chris"); msg.put("age", 18); msg.put("msgId", i); - sink.batchAdd(new Message(msg));//插入缓存,当条数>batchSize时,刷新存储 + sink.batchAdd(new Message(msg), null);//插入缓存,当条数>batchSize时,刷新存储 } sink.flush();//把缓存当数据刷新到存储 } @@ -52,7 +52,7 @@ public void testSinkAutoFlush() throws InterruptedException { msg.put("name", "chris"); msg.put("age", 18); msg.put("msgId", i); - sink.batchAdd(new Message(msg));//插入缓存,当条数>batchSize时,刷新存储 + sink.batchAdd(new Message(msg), null);//插入缓存,当条数>batchSize时,刷新存储 } while (true) { Thread.sleep(1000); diff --git a/rocketmq-streams-configurable/pom.xml b/rocketmq-streams-configurable/pom.xml index 12630a55..ab90f715 100755 --- a/rocketmq-streams-configurable/pom.xml +++ b/rocketmq-streams-configurable/pom.xml @@ -16,7 +16,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.rocketmq @@ -31,5 +33,10 @@ org.apache.rocketmq rocketmq-streams-serviceloader + + + org.apache.httpcomponents + httpclient + diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/ConfigurableComponent.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/ConfigurableComponent.java index 4f86cb64..c37571bf 100644 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/ConfigurableComponent.java +++ b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/ConfigurableComponent.java @@ -33,10 +33,12 @@ import org.apache.rocketmq.streams.configurable.service.ConfigurableServiceFactory; /** - * 对Configurable对象,做统一的管理,统一查询,插入和更新。 insert/update 把configuabel对象写入存储,支持文件存储(file),内存存储(memory)和db存储(DB)。可以在配置通过这个ConfigureFileKey.CONNECT_TYPE key 配置 query 是基于内存的查询,对象定时load到内存,可以在属性文件通过这个ConfigureFileKey.POLLING_TIME key配置加载周期,单位是秒 新对象加载后生效,已经存在的对象只有updateFlag发生变化才会被替换 + * 对Configurable对象,做统一的管理,统一查询,插入和更新。 insert/update 把configuabel对象写入存储, + * 支持文件存储(file),内存存储(memory)和db存储(DB)。可以在配置通过这个ConfigureFileKey.CONNECT_TYPE key 配置 + * query 是基于内存的查询,对象定时load到内存,可以在属性文件通过这个ConfigureFileKey.POLLING_TIME key配置加载周期, + * 单位是秒 新对象加载后生效,已经存在的对象只有updateFlag发生变化才会被替换 */ -public class ConfigurableComponent extends AbstractComponent - implements IConfigurableService { +public class ConfigurableComponent extends AbstractComponent implements IConfigurableService { private static final Log LOG = LogFactory.getLog(ConfigurableComponent.class); @@ -46,8 +48,7 @@ public class ConfigurableComponent extends AbstractComponent T queryConfigurable(String configurableType, String name) { return (T) queryConfigurableByIdent(configurableType, name); } - //protected void insertConfigurable(JSONObject message, IConfigurable configurable) { - // ConfigurableUtil.insertConfigurable(message, configurable, this.configureService); - //} - @Override public String getNamespace() { if (configureService instanceof AbstractConfigurableService) { diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractConfigurableService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractConfigurableService.java index bbfbd20c..37eb2786 100644 --- a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractConfigurableService.java +++ b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/AbstractConfigurableService.java @@ -20,7 +20,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; @@ -66,8 +65,7 @@ public AbstractConfigurableService(Properties properties) { public AbstractConfigurableService() { } - @Override - public IConfigurable queryConfigurableByIdent(String identification) { + @Override public IConfigurable queryConfigurableByIdent(String identification) { return name2ConfigurableMap.get(identification); } @@ -105,47 +103,44 @@ protected boolean equals(String key, List newConfigureList) { return false; } - @Override - public List queryConfigurableByType(String type) { + @Override public List queryConfigurableByType(String type) { List list = queryConfigurable(type); if (list == null) { return new ArrayList(); } - List result = new ArrayList(); + List result = new ArrayList<>(); for (IConfigurable configurable : list) { result.add((T) configurable); } return result; } - @Override - public boolean refreshConfigurable(String namespace) { - + @Override public boolean refreshConfigurable(String namespace) { this.namespace = namespace; Map> tempType2ConfigurableMap = new HashMap<>(); Map tempName2ConfigurableMap = new HashMap<>(); GetConfigureResult configures = loadConfigurable(namespace); if (configures != null && configures.isQuerySuccess() && configures.getConfigurables() != null) { List configurables = configures.getConfigurables(); - List configurableList = checkAndUpdateConfigurables(configurables, tempType2ConfigurableMap, tempName2ConfigurableMap); - for (IConfigurable configurable : configurableList) { - if (configurable instanceof IAfterConfigurableRefreshListener) { - ((IAfterConfigurableRefreshListener) configurable).doProcessAfterRefreshConfigurable(this); + if (configurables != null && !configurables.isEmpty()) { + checkAndUpdateConfigurables(configurables, tempType2ConfigurableMap, tempName2ConfigurableMap); + for (IConfigurable configurable : this.name2ConfigurableMap.values()) { + if (configurable instanceof IAfterConfigurableRefreshListener) { + ((IAfterConfigurableRefreshListener) configurable).doProcessAfterRefreshConfigurable(this); + } } } + return true; } return false; } - @Override - public T queryConfigurable(String configurableType, String name) { + @Override public T queryConfigurable(String configurableType, String name) { return (T) queryConfigurableByIdent(configurableType, name); } - protected List checkAndUpdateConfigurables(List configurables, - Map> tempType2ConfigurableMap, - Map tempName2ConfigurableMap) { + protected List checkAndUpdateConfigurables(List configurables, Map> tempType2ConfigurableMap, Map tempName2ConfigurableMap) { List configurableList = new ArrayList<>(); for (IConfigurable configurable : configurables) { try { @@ -164,9 +159,7 @@ protected List checkAndUpdateConfigurables(List co } private void destroyOldConfigurables(Map tempName2ConfigurableMap) { - Iterator> it = this.name2ConfigurableMap.entrySet().iterator(); - while (it.hasNext()) { - Map.Entry entry = it.next(); + for (Map.Entry entry : this.name2ConfigurableMap.entrySet()) { String key = entry.getKey(); IConfigurable value = entry.getValue(); if (!tempName2ConfigurableMap.containsKey(key)) { @@ -178,10 +171,9 @@ private void destroyOldConfigurables(Map tempName2Configu private void destroyOldConfigurable(IConfigurable oldConfigurable) { if (oldConfigurable instanceof AbstractConfigurable) { - ((AbstractConfigurable) oldConfigurable).destroy(); + oldConfigurable.destroy(); } - String key = getConfigureKey(oldConfigurable.getNameSpace(), oldConfigurable.getType(), - oldConfigurable.getConfigureName()); + String key = getConfigureKey(oldConfigurable.getNameSpace(), oldConfigurable.getType(), oldConfigurable.getConfigureName()); configurableMap.remove(key); } @@ -190,29 +182,26 @@ protected void initConfigurable(IConfigurable configurable) { AbstractConfigurable abstractConfigurable = (AbstractConfigurable) configurable; abstractConfigurable.setConfigurableService(this); } - configurable.init(); } private ScheduledExecutorService scheduledExecutorService; - @Override - public void initConfigurables(final String namespace) { + @Override public void initConfigurables(final String namespace) { refreshConfigurable(namespace); long polingTime = -1; if (this.properties != null) { String pollingTimeStr = this.properties.getProperty(AbstractComponent.POLLING_TIME); if (StringUtil.isNotEmpty(pollingTimeStr)) { - polingTime = Long.valueOf(pollingTimeStr); + polingTime = Long.parseLong(pollingTimeStr); } } if (polingTime > 0) { scheduledExecutorService = new ScheduledThreadPoolExecutor(3); scheduledExecutorService.scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { + @Override public void run() { try { refreshConfigurable(namespace); } catch (Exception e) { @@ -223,14 +212,12 @@ public void run() { } } - @Override - public List queryConfigurable(String type) { + @Override public List queryConfigurable(String type) { String key = MapKeyUtil.createKey(type); return type2ConfigurableMap.get(key); } - @Override - public IConfigurable queryConfigurableByIdent(String type, String name) { + @Override public IConfigurable queryConfigurableByIdent(String type, String name) { String key = MapKeyUtil.createKey(type, name); return name2ConfigurableMap.get(key); } @@ -244,8 +231,7 @@ public IConfigurable queryConfigurableByIdent(String type, String name) { */ protected abstract GetConfigureResult loadConfigurable(String namespace); - @Override - public void update(IConfigurable configurable) { + @Override public void update(IConfigurable configurable) { updateConfigurable(configurable); } @@ -253,8 +239,7 @@ public void update(IConfigurable configurable) { protected abstract void insertConfigurable(IConfigurable configurable); - protected boolean update(IConfigurable configurable, Map name2ConfigurableMap, - Map> type2ConfigurableMap) { + protected boolean update(IConfigurable configurable, Map name2ConfigurableMap, Map> type2ConfigurableMap) { if (configurable == null) { return false; } @@ -285,8 +270,7 @@ protected boolean update(IConfigurable configurable, Map return isUpdate; } - @Override - public void insert(IConfigurable configurable) { + @Override public void insert(IConfigurable configurable) { insertConfigurable(configurable); } @@ -323,16 +307,11 @@ public List queryConfiguableByNamespace(String... namespaces) { * @param configurable */ protected void put2Map(Map> map, String key, IConfigurable configurable) { - List list = map.get(key); - if (list == null) { - list = new ArrayList(); - map.put(key, list); - } + List list = map.computeIfAbsent(key, k -> new ArrayList<>()); list.add(configurable); } - @Override - public Collection findAll() { + @Override public Collection findAll() { return name2ConfigurableMap.values(); } @@ -356,8 +335,7 @@ protected Configure createConfigure(IConfigurable configurable) { return configure; } - @Override - public Map queryConfigurableMapByType(String type) { + @Override public Map queryConfigurableMapByType(String type) { List configurables = queryConfigurable(type); if (configurables == null) { return new HashMap(); @@ -426,9 +404,7 @@ protected IConfigurable convertConfigurable(Configure configure) { String className = null; try { String jsonString = configure.getJsonValue(); - IConfigurable configurable = - createConfigurableFromJson(configure.getNameSpace(), configure.getType(), configure.getName(), - jsonString); + IConfigurable configurable = createConfigurableFromJson(configure.getNameSpace(), configure.getType(), configure.getName(), jsonString); if (configurable instanceof Entity) { // add by wangtl 20171110 Configurable接口第三方包也在用,故不能Configurable里加接口,只能加到抽象类里,这里强转下 Entity abs = (Entity) configurable; @@ -453,25 +429,23 @@ protected IConfigurable convertConfigurable(Configure configure) { * * @param configurable */ - @SuppressWarnings("rawtypes") - protected void convertPost(IConfigurable configurable) { + @SuppressWarnings("rawtypes") protected void convertPost(IConfigurable configurable) { if (this.properties == null) { return; } - String identification = - MapKeyUtil.createKey(configurable.getNameSpace(), configurable.getType(), configurable.getConfigureName()); + String identification = MapKeyUtil.createKey(configurable.getNameSpace(), configurable.getType(), configurable.getConfigureName()); String propertyValue = this.properties.getProperty(identification); if (StringUtil.isEmpty(propertyValue)) { return; } String[] fieldName2Values = propertyValue.split(","); - if (fieldName2Values == null || fieldName2Values.length == 0) { + if (fieldName2Values.length == 0) { return; } for (String fieldName2Value : fieldName2Values) { try { String[] fieldName2ValueArray = fieldName2Value.split(":"); - if (fieldName2ValueArray == null || fieldName2ValueArray.length != 2) { + if (fieldName2ValueArray.length != 2) { continue; } String fieldName = fieldName2ValueArray[0]; @@ -486,9 +460,7 @@ protected void convertPost(IConfigurable configurable) { } catch (Exception e) { LOG.error("convert post error " + fieldName2Value, e); - continue; } - } } @@ -498,8 +470,7 @@ protected void convertPost(IConfigurable configurable) { * @param className class name * @return */ - @SuppressWarnings("rawtypes") - protected IConfigurable createConfigurable(String className) { + @SuppressWarnings("rawtypes") protected IConfigurable createConfigurable(String className) { return ReflectUtil.forInstance(className); } @@ -525,8 +496,7 @@ public void setConfigurables(List configurables) { } } - @Override - public String getNamespace() { + @Override public String getNamespace() { return namespace; } diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/FileConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileConfigureService.java similarity index 99% rename from rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/FileConfigureService.java rename to rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileConfigureService.java index 518ed09d..2d54a3e5 100644 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/FileConfigureService.java +++ b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileConfigureService.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.db.configuable; +package org.apache.rocketmq.streams.configurable.service.impl; import java.io.File; import java.util.ArrayList; diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/FileSupportParentConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileSupportParentConfigureService.java similarity index 96% rename from rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/FileSupportParentConfigureService.java rename to rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileSupportParentConfigureService.java index 7329a2f2..0b77b776 100644 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/FileSupportParentConfigureService.java +++ b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/FileSupportParentConfigureService.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.db.configuable; +package org.apache.rocketmq.streams.configurable.service.impl; import com.google.auto.service.AutoService; import java.util.Properties; diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpConfigureService.java new file mode 100644 index 00000000..a259a2fa --- /dev/null +++ b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpConfigureService.java @@ -0,0 +1,377 @@ +/* + * 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.rocketmq.streams.configurable.service.impl; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; +import java.io.UnsupportedEncodingException; +import java.nio.charset.Charset; +import java.nio.charset.UnsupportedCharsetException; +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSession; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.http.Consts; +import org.apache.http.Header; +import org.apache.http.HttpEntity; +import org.apache.http.ParseException; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.config.ConnectionConfig; +import org.apache.http.config.Registry; +import org.apache.http.config.RegistryBuilder; +import org.apache.http.conn.socket.ConnectionSocketFactory; +import org.apache.http.conn.socket.PlainConnectionSocketFactory; +import org.apache.http.conn.ssl.SSLConnectionSocketFactory; +import org.apache.http.entity.ContentType; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; +import org.apache.http.protocol.HTTP; +import org.apache.http.ssl.SSLContexts; +import org.apache.http.ssl.TrustStrategy; +import org.apache.http.util.Args; +import org.apache.http.util.CharArrayBuffer; +import org.apache.rocketmq.streams.common.component.AbstractComponent; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configurable.IConfigurable; +import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.utils.AESUtil; +import org.apache.rocketmq.streams.configurable.model.Configure; +import org.apache.rocketmq.streams.configurable.service.AbstractConfigurableService; + +public class HttpConfigureService extends AbstractConfigurableService { + + public static final int NORMAL_STATUES = 200; + public static final String CHARSET = "UTF-8"; + public static final int TIMOUT = 10000; + public static final int CONNECT_TIMOUT = 10000; + + protected String accessId; + protected String accessIdSecret; + protected String endPoint; + + protected transient CloseableHttpClient client; + + private static final Log LOG = LogFactory.getLog(HttpConfigureService.class); + + public static void main(String[] args) { + HttpConfigureService service = new HttpConfigureService("", "", "http://11.158.168.161:8888/queryConfigure"); + service.loadConfigurable("test"); + } + + public HttpConfigureService() { + init(); + } + + public HttpConfigureService(String accessId, String accessIdSecret, String endPoint) { + this.accessId = accessId; + this.accessIdSecret = accessIdSecret; + this.endPoint = endPoint; + init(); + } + + public HttpConfigureService(Properties properties) { + super(properties); + this.accessId = properties.getProperty(AbstractComponent.HTTP_AK); + this.accessIdSecret = properties.getProperty(AbstractComponent.HTTP_SK); + this.endPoint = properties.getProperty(HTTP_SERVICE_ENDPOINT) + "/queryConfigure"; + init(); + } + + public void init() { + RequestConfig.Builder configBuilder = RequestConfig.custom(); + configBuilder.setConnectionRequestTimeout(CONNECT_TIMOUT); + configBuilder.setConnectTimeout(CONNECT_TIMOUT); + configBuilder.setSocketTimeout(TIMOUT); + SSLConnectionSocketFactory sslsf = null; + try { + SSLContext sslcontext = SSLContexts.custom().loadTrustMaterial(new TrustStrategy() { + @Override + public boolean isTrusted(X509Certificate[] chain, String authType) throws CertificateException { + return true; + } + }).build(); + sslsf = new SSLConnectionSocketFactory(sslcontext, new HostnameVerifier() { + @Override + public boolean verify(String s, SSLSession sslSession) { + return true; + } + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + ConnectionConfig connectionConfig = ConnectionConfig.custom().setCharset(Consts.UTF_8).build(); + Registry socketFactoryRegistry = RegistryBuilder.create().register("https", + sslsf).register("http", + new PlainConnectionSocketFactory()).build(); + PoolingHttpClientConnectionManager connManager = new PoolingHttpClientConnectionManager(socketFactoryRegistry); + connManager.setDefaultConnectionConfig(connectionConfig); + connManager.setMaxTotal(500); + connManager.setDefaultMaxPerRoute(50); + HttpClientBuilder clientBuilder = HttpClients.custom(); + + clientBuilder.setDefaultRequestConfig(configBuilder.build()); + clientBuilder.setSSLSocketFactory(sslsf); + clientBuilder.setConnectionManager(connManager); + + client = clientBuilder.build(); + + } + + @Override + public GetConfigureResult loadConfigurable(String namespace) { + GetConfigureResult result = new GetConfigureResult(); +// request.setDipperNamespace(namespace); + try { + JSONObject param = new JSONObject(); + param.put("namespace", namespace); + CloseableHttpResponse response = post(endPoint, param.toJSONString(), null); + if (response == null) { + result.setQuerySuccess(false); + if (LOG.isErrorEnabled()) { + LOG.error("loadConfigurable error!namespace=" + namespace + ",response=" + JSONObject.toJSONString( + response)); + } + } else { + result.setQuerySuccess(true); + List configures = new ArrayList(); + String content = toString(response.getEntity(), Charset.forName(CHARSET)); + JSONObject object = JSONObject.parseObject(content); + JSONArray data = object.getJSONArray("data"); + System.out.println(data.size()); + configures = convert2Configure(data); + List configurables = convert(configures); + result.setConfigurables(configurables); + System.out.println(configures.size() + " " + configurables.size()); +// Data data = response.getEntity().getContent() +// if (data != null) { +// for (Item item : data.getItems()) { +// Configure configure = convert2Configure(item); +// configures.add(configure); +// } +// } +// result.setConfigurables(convert(configures)); + } + } catch (Exception e) { + result.setQuerySuccess(false); + if (LOG.isErrorEnabled()) { + LOG.error("loadConfigurable error!namespace=" + namespace, e); + } + } + return result; + } + + public CloseableHttpResponse get(String url, Map param) { + + try { + HttpGet httpGet = new HttpGet(url); + if (param != null && param.size() > 0) { + List
    headers = new ArrayList<>(); + for (Map.Entry tmp : param.entrySet()) { + httpGet.addHeader(tmp.getKey(), tmp.getValue()); + } + } + CloseableHttpResponse response = client.execute(httpGet); + return response; + } catch (Exception e) { + e.printStackTrace(); + } + return null; + + } + + public CloseableHttpResponse get(String url, Header... headers) { + try { + HttpGet httpGet = new HttpGet(url); + if (headers != null && headers.length > 0) { + for (Header header : headers) { + httpGet.addHeader(header); + } + } + CloseableHttpResponse response = client.execute(httpGet); + return response; + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } + + public CloseableHttpResponse post(String url, String body, Header... headers) { + try { + HttpPost httpPost = new HttpPost(url); + StringEntity stringEntity = new StringEntity(body, CHARSET); + stringEntity.setContentEncoding(CHARSET); + stringEntity.setContentType("application/json"); + httpPost.setEntity(stringEntity); + if (headers != null && headers.length > 0) { + for (Header header : headers) { + httpPost.addHeader(header); + } + } + CloseableHttpResponse response = client.execute(httpPost); + return response; + + } catch (Exception e) { + e.printStackTrace(); + } + return null; + } + + protected List convert2Configure(JSONArray array) { + List configures = new ArrayList(); + for (int i = 0; i < array.size(); i++) { + Configure configure = array.getObject(i, Configure.class); + if (!namespace.equalsIgnoreCase(configure.getNameSpace())) { + continue; + } + try { + configure.setJsonValue(AESUtil.aesDecrypt(configure.getJsonValue(), + ComponentCreator.getProperties().getProperty(ConfigureFileKey.SECRECY, ConfigureFileKey.SECRECY_DEFAULT))); + } catch (Exception e) { + e.printStackTrace(); + } + configures.add(configure); + } + return configures; + } + + protected List convert(List configures) { + if (configures == null) { + return new ArrayList(); + } + List configurables = new ArrayList(); + for (Configure configure : configures) { + IConfigurable configurable = convert(configure); + if (configurable != null) { + configurables.add(configurable); + } + } + return configurables; + } + +// private Configure convert2Configure(Item item) { +// Configure configure = new Configure(); +// configure.setAccountId(item.getAccountId()); +// configure.setAccountName(item.getAccountName()); +// configure.setAccountNickName(item.getAccountNickName()); +// configure.setClientIp(item.getDipperClientIp()); +// configure.setJsonValue(item.getJsonValue()); +// configure.setName(item.getName()); +// configure.setNameSpace(item.getDipperNamespace()); +// configure.setRequestId(item.getDipperRequestId()); +// configure.setType(item.getType()); +// return configure; +// } + + public String toString(HttpEntity entity, Charset defaultCharset) throws IOException, ParseException { + Args.notNull(entity, "Entity"); + InputStream instream = entity.getContent(); + if (instream == null) { + return null; + } else { + try { + Args.check(entity.getContentLength() <= 2147483647L, "HTTP entity too large to be buffered in memory"); + int i = (int) entity.getContentLength(); + if (i < 0) { + i = 4096; + } + + Charset charset = null; + + try { + ContentType contentType = ContentType.get(entity); + if (contentType != null) { + charset = contentType.getCharset(); + } + } catch (UnsupportedCharsetException var13) { + if (defaultCharset == null) { + throw new UnsupportedEncodingException(var13.getMessage()); + } + } + + if (charset == null) { + charset = defaultCharset; + } + + if (charset == null) { + charset = HTTP.DEF_CONTENT_CHARSET; + } + + Reader reader = new InputStreamReader(instream, charset); + CharArrayBuffer buffer = new CharArrayBuffer(i); + char[] tmp = new char[1024]; + + int l; + while ((l = reader.read(tmp)) != -1) { + buffer.append(tmp, 0, l); + } + + String var9 = buffer.toString(); + return var9; + } finally { + instream.close(); + } + } + } + + public String getAccessId() { + return accessId; + } + + public String getAccessIdSecret() { + return accessIdSecret; + } + + public void setAccessId(String accessId) { + this.accessId = accessId; + } + + public void setAccessIdSecret(String accessIdSecret) { + this.accessIdSecret = accessIdSecret; + } + + @Override + public List loadConfigurableFromStorage(String type) { + return null; + } + + @Override + protected void updateConfigurable(IConfigurable configurable) { + + } + + @Override + protected void insertConfigurable(IConfigurable configurable) { + + } +} diff --git a/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpSupportParentConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpSupportParentConfigureService.java new file mode 100644 index 00000000..9f9d3768 --- /dev/null +++ b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/HttpSupportParentConfigureService.java @@ -0,0 +1,44 @@ +/* + * 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.rocketmq.streams.configurable.service.impl; + +import com.google.auto.service.AutoService; +import java.util.Properties; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.configurable.IConfigurableService; +import org.apache.rocketmq.streams.common.model.ServiceName; +import org.apache.rocketmq.streams.configurable.service.AbstractSupportParentConfigureService; +import org.apache.rocketmq.streams.configurable.service.ConfigurableServcieType; + +@AutoService(IConfigurableService.class) +@ServiceName(ConfigurableServcieType.HTTP_SERVICE_NAME) +public class HttpSupportParentConfigureService extends AbstractSupportParentConfigureService { + + private static final Log LOG = LogFactory.getLog(HttpConfigureService.class); + + public HttpSupportParentConfigureService() { + + } + + @Override + protected void initBeforeInitConfigurable(Properties property) { + parentConfigureService = new HttpConfigureService(properties); + configureService = new HttpConfigureService(properties); + + } +} diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/MemoryConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemoryConfigureService.java similarity index 98% rename from rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/MemoryConfigureService.java rename to rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemoryConfigureService.java index 65a58de7..3de2e4d7 100644 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/MemoryConfigureService.java +++ b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemoryConfigureService.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.db.configuable; +package org.apache.rocketmq.streams.configurable.service.impl; import java.util.ArrayList; import java.util.HashMap; diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/MemorySupportParentConfigureService.java b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemorySupportParentConfigureService.java similarity index 96% rename from rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/MemorySupportParentConfigureService.java rename to rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemorySupportParentConfigureService.java index 6b777cb3..0c8765b2 100644 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/MemorySupportParentConfigureService.java +++ b/rocketmq-streams-configurable/src/main/java/org/apache/rocketmq/streams/configurable/service/impl/MemorySupportParentConfigureService.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.streams.db.configuable; +package org.apache.rocketmq.streams.configurable.service.impl; import com.google.auto.service.AutoService; import java.util.Properties; diff --git a/rocketmq-streams-db-operator/pom.xml b/rocketmq-streams-db-operator/pom.xml index 38b48fdd..fc5cb329 100755 --- a/rocketmq-streams-db-operator/pom.xml +++ b/rocketmq-streams-db-operator/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.rocketmq diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBConfigureService.java b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBConfigureService.java index e57bcf3a..234b4ab7 100644 --- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBConfigureService.java +++ b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/configuable/DBConfigureService.java @@ -134,12 +134,9 @@ protected List queryConfigureByNamespaceInner(String type, String... @Override public List queryConfiguableByNamespace(String... namespaces) { List configures = queryConfigureByNamespace(namespaces); - List configurables = convert(configures); - return configurables; + return convert(configures); } - - protected void saveOrUpdate(IConfigurable configure) { JDBCDriver jdbcDataSource = createResouce(); String sql = AbstractConfigurable.createSQL(configure, this.tableName); @@ -194,9 +191,9 @@ protected T getColumnValue(Map row, String columnName) { return null; } if (java.math.BigInteger.class.isInstance(value)) { - return (T)Long.valueOf(value.toString()); + return (T) Long.valueOf(value.toString()); } - return (T)value; + return (T) value; } @@ -271,7 +268,7 @@ public List loadConfigurableFromStorage(String type List configurables = convert(configures); List result = new ArrayList<>(); for (IConfigurable configurable : configurables) { - result.add((T)configurable); + result.add((T) configurable); } return result; } diff --git a/rocketmq-streams-examples/pom.xml b/rocketmq-streams-examples/pom.xml index dc1fc7f5..46fabf08 100644 --- a/rocketmq-streams-examples/pom.xml +++ b/rocketmq-streams-examples/pom.xml @@ -40,9 +40,20 @@ rocketmq-streams-clients - mysql - mysql-connector-java - 8.0.19 + com.alibaba + fastjson + + + junit + junit + + + org.apache.rocketmq + rocketmq-client + + + org.apache.rocketmq + rocketmq-streams-commons jar diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/ProducerFromFile.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/ProducerFromFile.java index 163d8116..0508c303 100644 --- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/ProducerFromFile.java +++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/ProducerFromFile.java @@ -19,6 +19,10 @@ package org.apache.rocketmq.streams.examples.send; +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.remoting.common.RemotingHelper; + import java.io.BufferedReader; import java.io.File; import java.io.FileReader; @@ -28,10 +32,6 @@ import java.util.List; import java.util.concurrent.atomic.AtomicLong; -import org.apache.rocketmq.client.producer.DefaultMQProducer; -import org.apache.rocketmq.common.message.Message; -import org.apache.rocketmq.remoting.common.RemotingHelper; - public class ProducerFromFile { private static final DefaultMQProducer producer = new DefaultMQProducer("test-group"); private static final AtomicLong count = new AtomicLong(0); diff --git a/rocketmq-streams-filter/pom.xml b/rocketmq-streams-filter/pom.xml index e273d094..ca00220b 100755 --- a/rocketmq-streams-filter/pom.xml +++ b/rocketmq-streams-filter/pom.xml @@ -15,7 +15,8 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java index 1d7ed404..53d748f4 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java @@ -28,6 +28,7 @@ import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.datatype.ListDataType; import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.model.NameCreatorContext; import org.apache.rocketmq.streams.common.utils.ContantsUtil; import org.apache.rocketmq.streams.common.utils.DataTypeUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; @@ -73,8 +74,7 @@ public static Rule createRule(String namespace, String ruleName, String expressi return rule; } - public static boolean executeExecute(String namespace, String expressionStr, IMessage message, - AbstractContext context) { + public static boolean executeExecute(String namespace, String expressionStr, IMessage message, AbstractContext context) { Rule rule = null; String key = expressionStr; rule = cache.get(key); @@ -320,8 +320,8 @@ public static String parseExpression(String namespace, String ruleName, String e expressionStr = ContantsUtil.doConstantReplace(expressionStr, flag2ExpressionStr, 1); } String relationStr = expressionStr; - NameCreator nameCreator = NameCreator.createOrGet(ruleName); - if (expressionStr.indexOf("(") == -1) { + NameCreator nameCreator = NameCreatorContext.get().createOrGet(ruleName); + if (!expressionStr.contains("(")) { relationStr = creator.createExpression(namespace, ruleName, expressionStr, expressions, containsContant, flag2ExpressionStr, nameCreator, relationStr); return relationStr; } @@ -339,27 +339,6 @@ public static String parseExpression(String namespace, String ruleName, String e } } return relationStr; - // for (int i = 0; i < expressionStr.length(); i++) { - // String word = expressionStr.substring(i, i + 1); - // - // if ("(".equals(word) ) { - // startExpression = true; - // continue; - // } - // if (")".equals(word)) { - // if (startExpression) { - // String expresionStr = expressionSb.toString(); - // relationStr=creator.createExpression(namespace,ruleName,expresionStr,expressions,containsContant,flag2ExpressionStr,nameCreator,relationStr); - // expressionSb = new StringBuilder(); - // } - // startExpression = false; - // continue; - // } - // if (startExpression) { - // expressionSb.append(word); - // } - // } - // return relationStr; } /** @@ -374,7 +353,7 @@ protected static String parseExpression(String namespace, String ruleName, Strin if (endIndex == -1) { return relationStr; } - if (relationStr.indexOf(",") == -1) { + if (!relationStr.contains(",")) { return relationStr; } for (int i = endIndex; i > 0; i--) { @@ -382,13 +361,13 @@ protected static String parseExpression(String namespace, String ruleName, Strin if (word.equals("(")) { String expressionStr = relationStr.substring(i, endIndex); String oriStr = "(" + expressionStr + ")"; - if (expressionStr.indexOf(",") != -1) { + if (expressionStr.contains(",")) { relationStr = creator.createExpression(namespace, ruleName, expressionStr, expressions, containsContant, contantsFlags, nameCreator, relationStr); relationStr = parseExpression(namespace, ruleName, relationStr, expressions, flag2ExpressionStr, creator, containsContant, nameCreator, contantsFlags); return relationStr; } else { - String relationFlag = NameCreator.createNewName("relation"); + String relationFlag = NameCreatorContext.get().createNewName("relation"); flag2ExpressionStr.put(relationFlag, oriStr); relationStr = relationStr.replace(oriStr, relationFlag); return parseExpression(namespace, ruleName, relationStr, expressions, flag2ExpressionStr, creator, containsContant, nameCreator, contantsFlags); @@ -399,19 +378,13 @@ protected static String parseExpression(String namespace, String ruleName, Strin } interface IRuleExpressionCreator { - - String createExpression(String namespace, String ruleName, String expresionStr, List expressions, - boolean containsContant, - Map flag2ExpressionStr, NameCreator nameCreator, String relationStr); + String createExpression(String namespace, String ruleName, String expresionStr, List expressions, boolean containsConstant, Map flag2ExpressionStr, NameCreator nameCreator, String relationStr); } protected static IRuleExpressionCreator ruleExpressionCreator = new IRuleExpressionCreator() { @Override - public String createExpression(String namespace, String ruleName, String expresionStr, - List expressions, boolean containsContant, Map flag2ExpressionStr, - NameCreator nameCreator, String relationStr) { - String[] values = ExpressionBuilder.createElement(expresionStr, containsContant, flag2ExpressionStr); - //expresionStr= ContantsUtil.restore(expresionStr,flag2ExpressionStr); + public String createExpression(String namespace, String ruleName, String expresionStr, List expressions, boolean containsConstant, Map flag2ExpressionStr, NameCreator nameCreator, String relationStr) { + String[] values = ExpressionBuilder.createElement(expresionStr, containsConstant, flag2ExpressionStr); String expressionName = nameCreator.createName(ruleName); SimpleExpression expression = null; @@ -428,7 +401,7 @@ public String createExpression(String namespace, String ruleName, String expresi if (expression != null) { expression.setNameSpace(namespace); expression.setConfigureName(expressionName); - if (relationStr.indexOf("(") == -1) { + if (!relationStr.contains("(")) { relationStr = relationStr.replace(expresionStr, expressionName); } else { relationStr = relationStr.replace("(" + expresionStr + ")", expressionName); diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java index 89a398e7..55190561 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java @@ -25,16 +25,18 @@ import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.common.monitor.IMonitor; -import org.apache.rocketmq.streams.common.monitor.TopologyFilterMonitor; import org.apache.rocketmq.streams.filter.function.expression.ExpressionFunction; import org.apache.rocketmq.streams.filter.operator.Rule; import org.apache.rocketmq.streams.filter.operator.action.Action; import org.apache.rocketmq.streams.filter.operator.expression.Expression; +import org.apache.rocketmq.streams.filter.operator.expression.RelationExpression; import org.apache.rocketmq.streams.filter.operator.var.Var; import org.apache.rocketmq.streams.script.function.model.FunctionConfigure; import org.apache.rocketmq.streams.script.function.service.impl.ScanFunctionService; import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; import java.util.Properties; import java.util.Vector; import java.util.concurrent.ConcurrentHashMap; @@ -101,7 +103,7 @@ public class RuleContext extends AbstractContext implements Serializabl private static volatile boolean initflag = false; - protected TopologyFilterMonitor expressionMonitor = new TopologyFilterMonitor(); + public static void initSuperRuleContext(ContextConfigure contextConfigure) { if (!initflag) { @@ -366,11 +368,27 @@ public void setRuleMonitor(IMonitor ruleMonitor) { this.ruleMonitor = ruleMonitor; } - public TopologyFilterMonitor getExpressionMonitor() { - return expressionMonitor; - } + public static void addNotFireExpressionMonitor( + Object expression,AbstractContext context) { + + if(RelationExpression.class.isInstance(expression)){ + List notFireExpressionMonitor=new ArrayList<>(); + RelationExpression relationExpression=(RelationExpression) expression; + for(String expressionName:notFireExpressionMonitor){ + if(!relationExpression.getValue().contains(expressionName)){ + notFireExpressionMonitor.add(expressionName); + } + } + notFireExpressionMonitor.add(relationExpression.getConfigureName()); + context.setNotFireExpressionMonitor(notFireExpressionMonitor); + }else if(Expression.class.isInstance(expression)) { + Expression e=(Expression)expression; + context.getNotFireExpressionMonitor().add(e.getConfigureName()); + }else if(String.class.isInstance(expression)){ + context.getNotFireExpressionMonitor().add((String)expression); + }else { + LOG.warn("can not support the express "+expression); + } - public void setExpressionMonitor(TopologyFilterMonitor expressionMonitor) { - this.expressionMonitor = expressionMonitor; } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/impl/DefaultRuleEngine.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/impl/DefaultRuleEngine.java index 40c98307..28948b3d 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/impl/DefaultRuleEngine.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/engine/impl/DefaultRuleEngine.java @@ -17,7 +17,10 @@ package org.apache.rocketmq.streams.filter.engine.impl; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.AbstractContext; @@ -25,16 +28,23 @@ import org.apache.rocketmq.streams.common.monitor.IMonitor; import org.apache.rocketmq.streams.common.monitor.TopologyFilterMonitor; import org.apache.rocketmq.streams.common.monitor.group.MonitorCommander; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; import org.apache.rocketmq.streams.common.utils.TraceUtil; import org.apache.rocketmq.streams.filter.context.RuleContext; import org.apache.rocketmq.streams.filter.engine.IRuleEngine; import org.apache.rocketmq.streams.filter.operator.Rule; import org.apache.rocketmq.streams.filter.operator.action.Action; +import org.apache.rocketmq.streams.filter.operator.expression.Expression; +import org.apache.rocketmq.streams.filter.operator.expression.RelationExpression; +import org.apache.rocketmq.streams.filter.optimization.dependency.CommonExpression; +import org.apache.rocketmq.streams.filter.optimization.dependency.SimplePipelineTree; +import org.apache.rocketmq.streams.filter.optimization.dependency.StateLessDependencyTree; +import org.apache.rocketmq.streams.script.service.IScriptExpression; public class DefaultRuleEngine implements IRuleEngine { private static final Log LOG = LogFactory.getLog(DefaultRuleEngine.class); - private static final Log RULEENGINE_MESSAGE_LOG = LogFactory.getLog("ruleengine_message"); @SuppressWarnings({"rawtypes", "unchecked"}) @@ -63,20 +73,13 @@ protected List executeRule(AbstractContext context, IMessage message, List return fireRules; } try { - boolean isTrace = TraceUtil.hit(message.getHeader().getTraceId()); for (Rule rule : excuteRules) { - RuleContext ruleContext = new RuleContext(message.getMessageBody(), rule); - if (context != null) { - context.syncSubContext(ruleContext); - } +// RuleContext ruleContext = new RuleContext(message.getMessageBody(), rule); +// if (context != null) { +// context.syncSubContext(ruleContext); +// } boolean isFireRule = rule.doMessage(message, context); - if (isFireRule == false && isTrace) { - TopologyFilterMonitor piplineExecutorMonitor = message.getHeader().getPiplineExecutorMonitor(); - if (piplineExecutorMonitor != null) { - piplineExecutorMonitor.setNotFireRule(rule.getConfigureName()); - piplineExecutorMonitor.setNotFireExpression2DependentFields(ruleContext.getExpressionMonitor().getNotFireExpression2DependentFields()); - } - } + if (isFireRule) { fireRules.add(rule); diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/CompareFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/CompareFunction.java index e0596a78..aa7941a4 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/CompareFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/CompareFunction.java @@ -20,6 +20,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.datatype.DataType; +import org.apache.rocketmq.streams.common.utils.DataTypeUtil; import org.apache.rocketmq.streams.common.utils.ReflectUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.filter.operator.expression.Expression; @@ -27,7 +29,7 @@ import org.apache.rocketmq.streams.script.utils.FunctionUtils; public abstract class CompareFunction extends AbstractExpressionFunction { - + public static String VAR_PREFIX="##*^%$#@!*";//标识一个表达式的值式一个变量,是一个特殊处理。这个标识会放到值的前面。如变量式uuid,值会变成&&&&##$$%^*uuid private static final Log LOG = LogFactory.getLog(CompareFunction.class); @Override @@ -39,6 +41,7 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E String varName = expression.getVarName(); Var var =expression.getVar(); varValue = var.doMessage(message, context); + /** * 两个数字比较的情况 */ @@ -61,9 +64,16 @@ public Boolean doExpressionFunction(IMessage message, AbstractContext context, E if (basicValue == null || basicVarValue == null) { return false; } + DataType dataType=expression.getDataType(); + if(String.class.isInstance(basicValue)&&basicValue.toString().startsWith(VAR_PREFIX)){ + String valueVarName=basicValue.toString().replace(VAR_PREFIX,""); + basicValue=message.getMessageBody().get(valueVarName); + dataType= DataTypeUtil.getDataTypeFromClass(varValue.getClass()); + basicVarValue=dataType.getData(varValue.toString()); + } - Class varClass = basicVarValue == null ? expression.getDataType().getDataClass() : basicVarValue.getClass(); - Class valueClass = basicValue == null ? expression.getDataType().getDataClass() : basicValue.getClass(); + Class varClass = basicVarValue == null ? dataType.getDataClass() : basicVarValue.getClass(); + Class valueClass = basicValue == null ? dataType.getDataClass() : basicValue.getClass(); try { match = (Boolean) ReflectUtil.invoke(this, "compare", new Class[] {varClass, valueClass}, diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java index ab7ed69f..5c75df17 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java @@ -38,13 +38,16 @@ import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.common.metadata.MetaData; import org.apache.rocketmq.streams.common.metadata.MetaDataField; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.ChainStage; import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; +import org.apache.rocketmq.streams.common.topology.metric.NotFireReason; import org.apache.rocketmq.streams.common.topology.model.AbstractRule; import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; import org.apache.rocketmq.streams.common.utils.TraceUtil; import org.apache.rocketmq.streams.filter.FilterComponent; +import org.apache.rocketmq.streams.filter.context.RuleContext; import org.apache.rocketmq.streams.filter.operator.action.Action; import org.apache.rocketmq.streams.filter.operator.action.impl.SinkAction; import org.apache.rocketmq.streams.filter.operator.expression.Expression; @@ -54,6 +57,9 @@ import org.apache.rocketmq.streams.filter.operator.var.InnerVar; import org.apache.rocketmq.streams.filter.operator.var.Var; import org.apache.rocketmq.streams.filter.optimization.ExpressionOptimization; +import org.apache.rocketmq.streams.filter.optimization.dependency.CommonExpression; +import org.apache.rocketmq.streams.filter.optimization.dependency.StateLessDependencyTree; +import org.apache.rocketmq.streams.script.service.IScriptExpression; public class Rule extends AbstractRule implements IAfterConfigurableRefreshListener, IStageBuilder { @@ -330,8 +336,53 @@ public void setMetaDataMap(Map metaDataMap) { @Override public Boolean doMessage(IMessage message, AbstractContext context) { boolean isTrace = TraceUtil.hit(message.getHeader().getTraceId()); + context.setNotFireExpressionMonitor(new ArrayList<>()); boolean isFireRule = processExpress(message, context, isTrace); + if (isFireRule == false && isTrace) { + NotFireReason notFireReason= context.getNotFireReason(); + if(notFireReason!=null){ + ChainPipeline chainPipeline=notFireReason.getPipeline(); + StateLessDependencyTree stateLessDependencyTree=new StateLessDependencyTree(chainPipeline); + List commonExpressions=stateLessDependencyTree.parseTopology(chainPipeline); + Map> filterFieldName2ETLScriptList=new HashMap<>(); + Map filterFieldName2OriFieldName=new HashMap<>(); + for(CommonExpression commonExpression:commonExpressions){ + String filterFieldName=commonExpression.getVarName(); + String origFieldName=commonExpression.getSourceVarName(); + filterFieldName2OriFieldName.put(filterFieldName,origFieldName); + List etlScript=filterFieldName2ETLScriptList.get(filterFieldName); + if(etlScript==null){ + etlScript=new ArrayList<>(); + filterFieldName2ETLScriptList.put(filterFieldName,etlScript); + } + for(IScriptExpression scriptExpression:commonExpression.getScriptExpressions()){ + if(!etlScript.contains(scriptExpression.toString())){ + etlScript.add(scriptExpression.toString()); + } + } + } + List filterFieldNames=new ArrayList<>(); + List expressions=new ArrayList<>(); + List expressionNames=context.getNotFireExpressionMonitor(); + for(String expressionName:expressionNames){ + Expression expression=this.getExpressionMap().get(expressionName); + if(expression==null){ + expressions.add(expressionName); + }else if(RelationExpression.class.isInstance(expression)){ + expressions.add(expression.toExpressionString(this.getExpressionMap())); + filterFieldNames.addAll(expression.getDependentFields(this.expressionMap)); + }else { + filterFieldNames.add(expression.getVarName()); + expressions.add(expression.toExpressionString(this.getExpressionMap())); + } + } + notFireReason.analysis(message,filterFieldName2ETLScriptList,filterFieldName2OriFieldName,expressions,filterFieldNames); + context.setNotFireReason(notFireReason); + } + + } + return isFireRule; } @@ -383,6 +434,10 @@ public String toMetaDataString() { return sb.toString(); } + @Override public String toString() { + return createOptimizationRule().toExpressionString(this.expressionMap); + } + public boolean isFinishVarAndExpression() { return isFinishVarAndExpression; } @@ -540,6 +595,9 @@ private boolean processExpress(IMessage message, AbstractContext context, boolea } boolean match = expression.doMessage(message, context); + if(!RelationExpression.class.isInstance(expression)){ + RuleContext.addNotFireExpressionMonitor(expression,context); + } if (!match) { return false; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/ExpressionRelationParser.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/ExpressionRelationParser.java index 070b21ca..506a88e9 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/ExpressionRelationParser.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/ExpressionRelationParser.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.model.NameCreatorContext; public class ExpressionRelationParser { public static final String OR = "|"; @@ -59,7 +60,7 @@ public static RelationExpression createRelations(String namespace, String ruleNa * @return */ private static RelationExpression createMixRelation(String namespace, String ruleName, String str, - List groupList) { + List groupList) { if (str.indexOf(OR) == -1) { return createSignleRelation(namespace, ruleName, str, AND, groupList); } @@ -83,9 +84,8 @@ private static RelationExpression createMixRelation(String namespace, String rul * @param sign * @return */ - private static RelationExpression createSignleRelation(String namespace, String ruleName, String str, String sign, - List groupList) { - NameCreator nameCreator = NameCreator.createOrGet(ruleName); + private static RelationExpression createSignleRelation(String namespace, String ruleName, String str, String sign, List groupList) { + NameCreator nameCreator = NameCreatorContext.get().createOrGet(ruleName); String expressionName = nameCreator.createName(ruleName); RelationExpression relationExpression = new RelationExpression(); relationExpression.setNameSpace(namespace); @@ -99,7 +99,6 @@ private static RelationExpression createSignleRelation(String namespace, String relationExpression.addExpression(value); } groupList.add(relationExpression); - return relationExpression; } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/ExpressionRelationPaser.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/ExpressionRelationPaser.java deleted file mode 100644 index 942ef67c..00000000 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/ExpressionRelationPaser.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.rocketmq.streams.filter.operator.expression; - -import java.util.ArrayList; -import java.util.List; -import org.apache.rocketmq.streams.common.model.NameCreator; - -public class ExpressionRelationPaser { - public static final String OR = "|"; - public static final String AND = "&"; - - - /** - * 创建表达式 - * - * @param namespace - * @param str - * @param groupList - * @return - */ - public static RelationExpression createRelations(String namespace,String ruleName, String str, List groupList) { - if (str.indexOf(")") == -1) { - return createMixRelation(namespace, ruleName,str, groupList); - } - int endIndex = str.indexOf(")"); - int startIndex = 0; - for (int i = endIndex; i > 0; i--) { - String word = str.substring(i - 1, i); - if ("(".equals(word)) { - startIndex = i; - break; - } - } - String expression = str.substring(startIndex, endIndex); - RelationExpression relationExpression = createMixRelation(namespace, ruleName,expression, groupList); - str = str.replace("(" + expression + ")", relationExpression.getConfigureName()); - return createRelations(namespace,ruleName, str, groupList); - } - - /** - * 混合表达关系处理 - * - * @param namespace - * @param str - * @return - */ - private static RelationExpression createMixRelation(String namespace,String ruleName, String str, - List groupList) { - if (str.indexOf(OR) == -1) { - return createSignleRelation(namespace,ruleName, str, AND, groupList); - } - String[] values = str.split("\\" + OR); - for (String value : values) { - String sign = AND; - if (value.indexOf(sign) != -1) { - RelationExpression relationExpression = createSignleRelation(namespace, ruleName, value, sign, groupList); - str = str.replace(value, relationExpression.getConfigureName()); - } - } - str = str.replace(" ", ""); - return createSignleRelation(namespace,ruleName, str, OR, groupList); - } - - /** - * 纯粹表达关系处理 - * - * @param namespace - * @param str - * @param sign - * @return - */ - private static RelationExpression createSignleRelation(String namespace,String ruleName, String str, String sign, - List groupList) { - NameCreator nameCreator=NameCreator.createOrGet(ruleName); - String expressionName=nameCreator.createName(ruleName); - RelationExpression relationExpression = new RelationExpression(); - relationExpression.setNameSpace(namespace); - relationExpression.setType(Expression.TYPE); - relationExpression.setValue(new ArrayList()); - relationExpression.setConfigureName(expressionName); - - relationExpression.setRelation(OR.equals(sign) ? "or" : "and"); - String[] values = str.split("\\" + sign); - for (String value : values) { - relationExpression.addExpression(value); - } - groupList.add(relationExpression); - - return relationExpression; - } -} diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpression.java index 33c7e4fc..5c0b9531 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpression.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpression.java @@ -30,6 +30,7 @@ import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.model.NameCreatorContext; import org.apache.rocketmq.streams.common.optimization.RegexEngine; import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; @@ -57,7 +58,7 @@ public GroupExpression(Rule rule, Var var, boolean isOrRelation) { this.var = var; this.varName = var.getVarName(); this.isOrRelation = isOrRelation; - this.setConfigureName(NameCreator.createNewName("expression.group")); + this.setConfigureName(NameCreatorContext.get().createNewName("expression.group")); value = new ArrayList<>(); this.setNameSpace(rule.getNameSpace()); fingerprintCache = FingerprintCache.getInstance(); diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/RelationExpression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/RelationExpression.java index 174b9aed..9dd3e28d 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/RelationExpression.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/RelationExpression.java @@ -113,7 +113,6 @@ public boolean supportQuickMatch(Expression expression, RuleContext context, Rul @Override public Boolean doMessage(IMessage message, AbstractContext context) { - Iterator it = iterator(); /** * 如果表达式组的值为空,则返回false @@ -136,11 +135,9 @@ public Boolean doMessage(IMessage message, AbstractContext context) { Boolean result = message.getMessageBody().getBoolean(expressionName); if (result != null) { if (result == false) { -// if(isTrace){ -// TopologyFilterMonitor piplineExecutorMonitor = new TopologyFilterMonitor(); -// piplineExecutorMonitor.addNotFireExpression(expressionName, expressionName); -// context.setExpressionMonitor(piplineExecutorMonitor); -// } + if(isTrace){ + RuleContext.addNotFireExpressionMonitor(expressionName,context); + } return optimizate(expressionName, false); } else { @@ -168,11 +165,10 @@ public Boolean doMessage(IMessage message, AbstractContext context) { return false; } if (!flag) { -// if(isTrace){ -// TopologyFilterMonitor piplineExecutorMonitor = new TopologyFilterMonitor(); -// piplineExecutorMonitor.addNotFireExpression(exp.toString(), exp.getDependentFields(rule.getExpressionMap())); -// context.setExpressionMonitor(piplineExecutorMonitor); -// } + + if(isTrace){ + RuleContext.addNotFireExpressionMonitor(exp,context); + } return optimizate(expressionName, false); } @@ -182,10 +178,7 @@ public Boolean doMessage(IMessage message, AbstractContext context) { return true; } else {// or flag = false; - TopologyFilterMonitor piplineExecutorMonitor =null; - if(isTrace){ - piplineExecutorMonitor= new TopologyFilterMonitor(); - } + while (it.hasNext()) { String expressionName = it.next(); @@ -210,12 +203,6 @@ public Boolean doMessage(IMessage message, AbstractContext context) { } else { //如果关系表达式未触发,则检测context中,有没有因为and失败的条件 -// if(isTrace){ -// if (context.getExpressionMonitor() != null && context.getExpressionMonitor().getNotFireExpression2DependentFields().size() > 0) { -// piplineExecutorMonitor.addNotFireExpression(context.getExpressionMonitor().getNotFireExpression2DependentFields()); -// } -// } - } } else { @@ -232,9 +219,9 @@ public Boolean doMessage(IMessage message, AbstractContext context) { } } -// if(isTrace){ -// context.setExpressionMonitor(piplineExecutorMonitor); -// } + if(isTrace){ + RuleContext.addNotFireExpressionMonitor(this,context); + } return false; } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/PiplineLogFingerprintAnalysis.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/PiplineLogFingerprintAnalysis.java index a94c0499..0271ba9f 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/PiplineLogFingerprintAnalysis.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/PiplineLogFingerprintAnalysis.java @@ -29,7 +29,6 @@ import org.apache.rocketmq.streams.common.topology.model.AbstractStage; import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; -import org.apache.rocketmq.streams.common.topology.stages.UnionChainStage; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder; import org.apache.rocketmq.streams.script.utils.FunctionUtils; @@ -136,10 +135,7 @@ private AbstractStage analysisStage(AbstractStage currentStage, Set logF } else if (ScriptChainStage.class.isInstance(currentStage)) { //continue - } else if (UnionChainStage.class.isInstance(currentStage)) { - return prewFilterStage; - - } else { + } else { return prewFilterStage; } AbstractStage nextStage = getNextStage(currentStage); diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/BlinkRuleV2Expression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/BlinkRuleV2Expression.java index dfac9994..ba90e359 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/BlinkRuleV2Expression.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/BlinkRuleV2Expression.java @@ -35,6 +35,7 @@ import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.model.NameCreator; +import org.apache.rocketmq.streams.common.model.NameCreatorContext; import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.FileUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; @@ -188,9 +189,9 @@ public Rule createRuleExpression() { RelationExpression relationExpression = new RelationExpression(); relationExpression.setValue(new ArrayList()); relationExpression.setRelation("and"); - relationExpression.setConfigureName(NameCreator.createNewName("_blink_rule_v2", "relation")); + relationExpression.setConfigureName(NameCreatorContext.get().createNewName("_blink_rule_v2", "relation")); for (Expression expression : expressions) { - String name = NameCreator.createNewName("_blink_rule_v2", expression.getVarName()); + String name = NameCreatorContext.get().createNewName("_blink_rule_v2", expression.getVarName()); expression.setConfigureName(name); expression.setNameSpace("tmp"); relationExpression.getValue().add(expression.getConfigureName()); diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/DependencyTree.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/DependencyTree.java index 46297c26..4881bdfd 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/DependencyTree.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/DependencyTree.java @@ -35,9 +35,11 @@ * raverse the pipeline to create a prefix filter fingerprint */ public class DependencyTree { + + + protected ChainPipeline chainPipeline; protected FingerprintCache fingerprintCache; - public DependencyTree(ChainPipeline pipeline, FingerprintCache fingerprintCache) { this.fingerprintCache = fingerprintCache; this.chainPipeline = pipeline; @@ -61,7 +63,7 @@ public List parse() { for (Map fingerprintMap : preFingerprintMap) { for (PreFingerprint fingerprint : fingerprintMap.values()) { fingerprint.getFilterChainStage().setPreFingerprint(fingerprint); - for (FilterChainStage previewFilterChainStage : fingerprint.getAllPreviewFilterChainStage()) { + for (AbstractStage previewFilterChainStage : fingerprint.getAllPreviewFilterChainStage()) { previewFilterChainStage.setPreFingerprint(fingerprint); } } @@ -84,12 +86,18 @@ public List parse() { * @param pipeline */ public List parseTopology(ChainPipeline pipeline) { + if(StateLessDependencyTree.cache.containsKey(pipeline)){ + return StateLessDependencyTree.cache.get(pipeline); + } List nextLalbes = pipeline.getChannelNextStageLabel(); List commonExpressions = new ArrayList<>(); parseTree(null, nextLalbes, pipeline, commonExpressions); + StateLessDependencyTree.cache.put(chainPipeline,commonExpressions); return commonExpressions; } + + /** * @param parentTreeNode * @param nextLables @@ -144,7 +152,7 @@ protected void parseTree(TreeNode parentTreeNode, List nextLables, Chain * @return */ protected boolean mergePreFingerprint(PreFingerprint fingerprint, ChainPipeline pipeline) { - String sourceLable = fingerprint.getSourceStageLable(); + String sourceLable = fingerprint.getSourceStageLabel(); if (sourceLable == null) { sourceLable = pipeline.getChannelName(); } @@ -152,11 +160,11 @@ protected boolean mergePreFingerprint(PreFingerprint fingerprint, ChainPipeline Map preFingerprintMap = preFingerprintExecutor.get(sourceLable); if (preFingerprintMap == null) { preFingerprintMap = new HashMap<>(); - preFingerprintMap.put(fingerprint.getNextStageLable(), fingerprint); + preFingerprintMap.put(fingerprint.getNextStageLabel(), fingerprint); preFingerprintExecutor.put(sourceLable, preFingerprintMap); return true; } - PreFingerprint previewPreFingerprint = preFingerprintMap.get(fingerprint.getNextStageLable()); + PreFingerprint previewPreFingerprint = preFingerprintMap.get(fingerprint.getNextStageLabel()); if (previewPreFingerprint != null && !mergeFingerprint(previewPreFingerprint, fingerprint)) { return false; } @@ -164,7 +172,7 @@ protected boolean mergePreFingerprint(PreFingerprint fingerprint, ChainPipeline fingerprint.addPreviwFilterChainStage(previewPreFingerprint.getAllPreviewFilterChainStage()); fingerprint.addPreviwFilterChainStage(previewPreFingerprint.getFilterChainStage()); } - preFingerprintMap.put(fingerprint.getNextStageLable(), fingerprint); + preFingerprintMap.put(fingerprint.getNextStageLabel(), fingerprint); return true; } @@ -174,7 +182,7 @@ protected boolean mergePreFingerprint(PreFingerprint fingerprint, ChainPipeline * * @return */ - private boolean mergeFingerprint(PreFingerprint preview, PreFingerprint current) { + protected boolean mergeFingerprint(PreFingerprint preview, PreFingerprint current) { Set previewLogFingerFieldNameSet = loadLogFingerFieldNames(preview); Set currentLogFingerFieldNameSet = loadLogFingerFieldNames(current); boolean inPrew = true; diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/SimplePipelineTree.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/SimplePipelineTree.java index 622ff2fc..6e26bc09 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/SimplePipelineTree.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/SimplePipelineTree.java @@ -17,12 +17,13 @@ package org.apache.rocketmq.streams.filter.optimization.dependency; import java.util.List; +import java.util.Map; import java.util.Set; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.model.Pipeline; import org.apache.rocketmq.streams.script.service.IScriptExpression; public class SimplePipelineTree { - protected Pipeline pipeline; protected TreeNode rootNode; protected TreeNode leafNode; diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/StateLessDependencyTree.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/StateLessDependencyTree.java new file mode 100644 index 00000000..aedf8f97 --- /dev/null +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/dependency/StateLessDependencyTree.java @@ -0,0 +1,84 @@ +/* + * 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.rocketmq.streams.filter.optimization.dependency; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.rocketmq.streams.common.optimization.fingerprint.FingerprintCache; +import org.apache.rocketmq.streams.common.optimization.fingerprint.PreFingerprint; +import org.apache.rocketmq.streams.common.topology.ChainPipeline; +import org.apache.rocketmq.streams.common.topology.model.AbstractStage; +import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage; +import org.apache.rocketmq.streams.common.topology.stages.ScriptChainStage; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; + +/** + * raverse the pipeline to create a prefix filter fingerprint + */ +public class StateLessDependencyTree extends DependencyTree{ + public static Map> cache=new HashMap<>(); + + + protected ChainPipeline chainPipeline; + protected Map> preFingerprintExecutor=new HashMap<>(); + public StateLessDependencyTree(ChainPipeline pipeline) { + super(pipeline,null); + this.chainPipeline = pipeline; + } + + + /** + * If the two pre filters are one branch, merge and replace the previous one with the latter one + * The consolidation condition is that the following branches have no new fingerprint fields or more filtering conditions + * + * @param fingerprint + * @param pipeline + * @return + */ + @Override + protected boolean mergePreFingerprint(PreFingerprint fingerprint, ChainPipeline pipeline) { + String sourceLable = fingerprint.getSourceStageLabel(); + if (sourceLable == null) { + sourceLable = pipeline.getChannelName(); + } + Map> preFingerprintExecutor = this.preFingerprintExecutor; + Map preFingerprintMap = preFingerprintExecutor.get(sourceLable); + if (preFingerprintMap == null) { + preFingerprintMap = new HashMap<>(); + preFingerprintMap.put(fingerprint.getNextStageLabel(), fingerprint); + preFingerprintExecutor.put(sourceLable, preFingerprintMap); + return true; + } + PreFingerprint previewPreFingerprint = preFingerprintMap.get(fingerprint.getNextStageLabel()); + if (previewPreFingerprint != null && !mergeFingerprint(previewPreFingerprint, fingerprint)) { + return false; + } + if (previewPreFingerprint != null) { + fingerprint.addPreviwFilterChainStage(previewPreFingerprint.getAllPreviewFilterChainStage()); + fingerprint.addPreviwFilterChainStage(previewPreFingerprint.getFilterChainStage()); + } + preFingerprintMap.put(fingerprint.getNextStageLabel(), fingerprint); + return true; + } + + +} diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousCompute.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousCompute.java index 438d1149..f9b2f16d 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousCompute.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousCompute.java @@ -74,7 +74,7 @@ public HomologousCompute(List commonExpressions, int cacheSize this.commonExpressions = commonExpressions; Map groupBySourceVarName = groupBySourceVarName(commonExpressions); this.groupBySourceVarName = groupBySourceVarName; - registHyperscan(groupBySourceVarName); +// registHyperscan(groupBySourceVarName); createExpressionIndexAndHomologousVar(groupBySourceVarName); fingerprintCache = new FingerprintCache(cacheSize); } @@ -313,4 +313,7 @@ public int size() { } } + public Map getGroupBySourceVarName() { + return groupBySourceVarName; + } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousOptimization.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousOptimization.java index 2bc61e89..f5542a14 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousOptimization.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/homologous/HomologousOptimization.java @@ -19,6 +19,7 @@ import com.alibaba.fastjson.JSONObject; import com.google.auto.service.AutoService; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.rocketmq.streams.common.context.AbstractContext; @@ -34,9 +35,10 @@ @AutoService(IHomologousOptimization.class) public class HomologousOptimization implements IHomologousOptimization { protected transient HomologousCompute homologousCompute; + protected static Map homologousComputeCache=new HashMap<>(); @Override - public void optimizate(List pipelines, int cacheSize, int preFingerprintCacheSize) { + public void optimizate(List> pipelines, int cacheSize, int preFingerprintCacheSize) { List commonExpressions = new ArrayList<>(); FingerprintCache fingerprintCache = new FingerprintCache(preFingerprintCacheSize); for (ChainPipeline pipeline : pipelines) { @@ -48,8 +50,17 @@ public void optimizate(List pipelines, int cacheSize, int preFing printOptimizatePipeline(pipeline); } homologousCompute = new HomologousCompute(commonExpressions, cacheSize); + for(ChainPipeline chainPipeline:pipelines){ + homologousComputeCache.put(chainPipeline,homologousCompute); + } } + + public HomologousCompute getHomologousCompute(ChainPipeline chainPipeline){ + return homologousComputeCache.get(chainPipeline); + } + + @Override public void calculate(IMessage message, AbstractContext context) { homologousCompute.calculate(message, context); diff --git a/rocketmq-streams-script/pom.xml b/rocketmq-streams-script/pom.xml index bd156399..7cd2b300 100755 --- a/rocketmq-streams-script/pom.xml +++ b/rocketmq-streams-script/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.rocketmq diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/LastValueAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/LastValueAccumulator.java new file mode 100644 index 00000000..987cc73d --- /dev/null +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/LastValueAccumulator.java @@ -0,0 +1,67 @@ +/* + * 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.rocketmq.streams.script.function.aggregation; + +import java.util.Iterator; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.script.annotation.Function; +import org.apache.rocketmq.streams.script.annotation.UDAFFunction; +import org.apache.rocketmq.streams.script.service.IAccumulator; + +@Function +@UDAFFunction("LAST_VALUE") +public class LastValueAccumulator implements IAccumulator { + + public static class LastValue { + public T value; + } + + @Override + public LastValue createAccumulator() { + return new LastValue(); + } + + @Override + public T getValue(LastValue accumulator) { + return (T) accumulator.value; + } + + @Override + public void accumulate(LastValue accumulator, Object... parameters) { + if (CollectionUtil.isEmpty(parameters) || parameters[0] == null) { + return; + } + accumulator.value = parameters[0]; + } + + @Override + public void merge(LastValue accumulator, Iterable its) { + Iterator it = its.iterator(); + while (it.hasNext()) { + LastValue next = it.next(); + if (next != null && next.value != null) { + accumulator.value = next.value; + } + } + } + + @Override + public void retract(LastValue accumulator, String... parameters) { + //TODO + } + +} diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/distinct/DistinctFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/distinct/DistinctFunction.java index a9cd6279..6a35b510 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/distinct/DistinctFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/distinct/DistinctFunction.java @@ -37,7 +37,6 @@ public boolean containsOrPut(String... keys) { } } } - cache = new KeySet(MAX_SIZE); String key = MapKeyUtil.createKey(keys); boolean success = cache.contains(key); if (!success) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonCreatorFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonCreatorFunction.java index 7a85b942..800fc3ce 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonCreatorFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonCreatorFunction.java @@ -32,7 +32,7 @@ public class JsonCreatorFunction { @FunctionMethod(value = "json_merge", alias = "jsonMerge", comment = "根据字段来组合json") public String extraJsonByField(IMessage message, FunctionContext context, @FunctionParamter(value = "json", comment = "字段名列表") String jsonFieldName) { - jsonFieldName = FunctionUtils.getValueString(message, context, jsonFieldName); + jsonFieldName = FunctionUtils.getConstant( jsonFieldName); JSONObject msg = message.getMessageBody().getJSONObject(jsonFieldName); message.getMessageBody().putAll(msg); return null; @@ -177,7 +177,7 @@ public String removeElement(IMessage message, FunctionContext context, @FunctionMethod(value = "jsonExpand", alias = "json_expand", comment = "展开一个json中的json") public void expandElement(IMessage message, FunctionContext context, @FunctionParamter(value = "array", comment = "代表要移除key的字段名或常量列表") String jsonSubFieldName) { - jsonSubFieldName = FunctionUtils.getValueString(message, context, jsonSubFieldName); + jsonSubFieldName = FunctionUtils.getConstant(jsonSubFieldName); String jsonValue = message.getMessageBody().getString(jsonSubFieldName); if (jsonValue == null) { return; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/UDTFFieldNameFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/UDTFFieldNameFunction.java new file mode 100644 index 00000000..621aefc0 --- /dev/null +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/UDTFFieldNameFunction.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.script.function.impl.json; + + +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.script.ScriptComponent; +import org.apache.rocketmq.streams.script.annotation.Function; +import org.apache.rocketmq.streams.script.annotation.FunctionMethod; +import org.apache.rocketmq.streams.script.annotation.FunctionParamter; +import org.apache.rocketmq.streams.script.context.FunctionContext; +import org.apache.rocketmq.streams.script.utils.FunctionUtils; + +@Function +public class UDTFFieldNameFunction { + + @FunctionMethod(value = "addAliasForNewField",comment = "获取msg中的json数据") + public Object addAliasForNewField(IMessage message, FunctionContext context, + @FunctionParamter(value = "string", comment = "代表json的字段名或常量") String fieldName,String alias,int i){ + fieldName=FunctionUtils.getConstant(fieldName); + alias=FunctionUtils.getConstant(alias); + Object object=message.getMessageBody().get(fieldName); + if(message.getMessageBody().containsKey("f"+i)) { + object=message.getMessageBody().get("f"+i); + message.getMessageBody().put(alias+fieldName,object); + } + else if(message.getMessageBody().containsKey(fieldName)&&!message.getMessageBody().containsKey("f"+i)){ + message.getMessageBody().put(alias+fieldName,object); + } + return object; + } + + +} diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/GrokFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/GrokFunction.java index ea93a841..babbe393 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/GrokFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/GrokFunction.java @@ -43,9 +43,9 @@ public JSONObject doGrok(IMessage message, AbstractContext context, String field 传入自定义的pattern, 会从已注册的patterns里面进行配对, 例如: TIMESTAMP_ISO8601:timestamp1, TIMESTAMP_ISO8601在注册的 patterns里面有对应的解析格式, 配对成功后, 会在match时按照固定的解析格式将解析结果存入map中, 此处timestamp1作为输出的key */ - grokStr = FunctionUtils.getValueString(message, context, grokStr); + grokStr = FunctionUtils.getConstant( grokStr); Grok grok = grokCompiler.compile(grokStr); - fieldName = FunctionUtils.getValueString(message, context, fieldName); + fieldName = FunctionUtils.getConstant(fieldName); String logMsg = message.getMessageBody().getString(fieldName); // 通过match()方法进行匹配, 对log进行解析, 按照指定的格式进行输出 Match grokMatch = grok.match(logMsg); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/Paser2JsonFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/Paser2JsonFunction.java index ebd4ad89..767153be 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/Paser2JsonFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/Paser2JsonFunction.java @@ -28,6 +28,7 @@ import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.context.FunctionContext; +import org.apache.rocketmq.streams.script.utils.FunctionUtils; @Function public class Paser2JsonFunction { @@ -40,23 +41,25 @@ public class Paser2JsonFunction { * @param context */ @FunctionMethod(value = "spread_json", alias = "autoJson", comment = "原始数据是嵌套json或jsonArray调用此方法会自动展开成单层") - public void spread2Json(IMessage message, FunctionContext context) { - JSONObject jsonObject = null; - if (message.isJsonMessage()) { - jsonObject = message.getMessageBody(); - } + public void spread2Json(IMessage message, FunctionContext context,String fieldName) { + fieldName= FunctionUtils.getConstant(fieldName); + + JSONObject jsonObject = message.getMessageBody().getJSONObject(fieldName); +// if (message.isJsonMessage()) { +// jsonObject = message.getMessageBody(); +// } List jsonArrayNames = new ArrayList<>(); jsonObject = spreadJson(jsonObject, jsonArrayNames); if (jsonArrayNames.size() > 0) { List jsonObjects = spreadJsonArray(jsonObject, jsonArrayNames); for (JSONObject tmp : jsonObjects) { IMessage copyMessage = message.copy(); - copyMessage.setMessageBody(tmp); + copyMessage.getMessageBody().putAll(tmp); context.getSplitMessages().add(copyMessage); } context.openSplitModel(); } else { - message.setMessageBody(jsonObject); + message.getMessageBody().putAll(jsonObject); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/PaserBySplitFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/PaserBySplitFunction.java index dc1ae439..219e6f2e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/PaserBySplitFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/PaserBySplitFunction.java @@ -28,6 +28,7 @@ import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.annotation.FunctionParamter; import org.apache.rocketmq.streams.script.context.FunctionContext; +import org.apache.rocketmq.streams.script.function.model.FunctionType; import org.apache.rocketmq.streams.script.utils.FunctionUtils; @Function @@ -37,14 +38,13 @@ public class PaserBySplitFunction { @FunctionMethod(value = "paserByComma", comment = "根据英文逗号分割字符串") public JSONObject paserByComma(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String... keyNames) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName) { String log = FunctionUtils.getValueString(message, context, fieldName); Map flags = (Map)context.get(CONST_MAP_KEY); if (flags == null) { flags = new HashMap<>(); } - return parseBySplit(message, context, log, fieldName, ",", flags, keyNames); + return parseBySplit(message, context, log, fieldName, ",", flags); } /** @@ -53,21 +53,19 @@ public JSONObject paserByComma(IMessage message, FunctionContext context, * @param message * @param context * @param asciiDec 十进制的ascii码 - * @param keyNames * @return */ - @FunctionMethod(value = "paserByAsciiSplit", comment = "根据char分割字符串,其中char通过ascii码转换过来,常用于使用不可见字符做分割") + @FunctionMethod(value = "paserBySign", comment = "根据char分割字符串,其中char通过ascii码转换过来,常用于使用不可见字符做分割") public JSONObject paserByAsciiSplit(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, - @FunctionParamter(value = "string", comment = "代表分割符") String asciiDec, - @FunctionParamter(value = "string", comment = "预计的字段名称") String... keyNames) { + @FunctionParamter(value = "string", comment = "代表分割符") String asciiDec) { char splitSign = (char)Integer.parseInt(asciiDec); String log = FunctionUtils.getValueString(message, context, fieldName); Map flags = (Map)context.get(CONST_MAP_KEY); if (flags == null) { flags = new HashMap<>(); } - return parseBySplit(message, context, log, fieldName, String.valueOf(splitSign), flags, keyNames); + return parseBySplit(message, context, log, fieldName, String.valueOf(splitSign), flags); } @FunctionMethod(value = "split", alias = "paserBySplit", comment = "通过分割符来进行日志解析") @@ -76,8 +74,7 @@ public JSONObject parseBySign(IMessage message, FunctionContext context, @FunctionParamter(value = "boolean", comment = "是否需要预先处理带括号的数据") boolean needBacket, @FunctionParamter(value = "boolean", comment = "是否预先处理时间类型的数据") boolean needDate, @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, - @FunctionParamter(value = "string", comment = "代表分割符") String sign, - @FunctionParamter(value = "string", comment = "预计的字段名称") String... names) { + @FunctionParamter(value = "string", comment = "代表分割符") String sign) { Map flags = new HashMap<>(); fieldName = FunctionUtils.getValueString(message, context, fieldName); String log = FunctionUtils.getValueString(message, context, fieldName); @@ -91,21 +88,20 @@ public JSONObject parseBySign(IMessage message, FunctionContext context, log = LogParserUtil.parseDate(log, flags); } sign = FunctionUtils.getValueString(message, context, sign); - return parseBySplit(message, context, log, fieldName, sign, flags, names); + return parseBySplit(message, context, log, fieldName, sign, flags); } @FunctionMethod(value = "split", alias = "paserBySplit", comment = "通过分割符来进行日志解析") public JSONObject parseBySign(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, - @FunctionParamter(value = "string", comment = "代表分割符") String sign, - @FunctionParamter(value = "string", comment = "预计的字段名称") String... names) { - sign = FunctionUtils.getValueString(message, context, sign); - String log = FunctionUtils.getValueString(message, context, fieldName); + @FunctionParamter(value = "string", comment = "代表分割符") String sign) { + sign = FunctionUtils.getConstant(sign); + String log = FunctionUtils.getValueString(message,context,fieldName); Map flags = (Map)context.get(CONST_MAP_KEY); if (flags == null) { flags = new HashMap<>(); } - return parseBySplit(message, context, log, fieldName, sign, flags, names); + return parseBySplit(message, context, log, fieldName, sign, flags); } /** @@ -135,28 +131,16 @@ public JSONObject parseBySign(IMessage message, FunctionContext context, * @param fieldName 字段名称 * @param sign 分割符号 * @param flags 常量和原始值的映射 - * @param names 预计的字段名称,这里的名称未做常量处理 * @return */ - private JSONObject parseBySplit(IMessage message, FunctionContext context, String log, String fieldName, String sign, Map flags, String... names) { + private JSONObject parseBySplit(IMessage message, FunctionContext context, String log, String fieldName, String sign, Map flags) { if (signs.containsKey(sign)) { sign = signs.get(sign); } String[] values = log.split(sign); Map jsonObject = new HashMap(); for (int i = 0; i < values.length; i++) { - String name = null; - if (names != null && names.length > i) { - name = names[i]; - if (StringUtil.isEmpty(name)) { - name = fieldName + i; - } else { - name = FunctionUtils.getValueString(message, context, name); - } - - } else { - name = fieldName + i; - } + String name = FunctionType.UDTF.getName() +i; String value = values[i]; String tmp = flags.get(value); if (StringUtil.isNotEmpty(tmp)) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/RegexParserFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/RegexParserFunction.java index a23a7c0c..18df0581 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/RegexParserFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/parser/RegexParserFunction.java @@ -27,6 +27,7 @@ import org.apache.rocketmq.streams.script.annotation.FunctionMethod; import org.apache.rocketmq.streams.script.annotation.FunctionParamter; import org.apache.rocketmq.streams.script.context.FunctionContext; +import org.apache.rocketmq.streams.script.function.model.FunctionType; import org.apache.rocketmq.streams.script.utils.FunctionUtils; @Function @@ -36,13 +37,11 @@ public class RegexParserFunction { @FunctionMethod(value = "paserByRegex", comment = "通过正则解析实例日志") public String paserByRegex(IMessage message, FunctionContext context, @FunctionParamter(value = "string", comment = "代表字符串的字段名") String fieldName, - @FunctionParamter(value = "string", comment = "正则表达式") String regex, - @FunctionParamter(value = "string", comment = "正则表达式中的字段") String... keyNames) { - if (message.isJsonMessage()) { - return message.getMessageBody().toJSONString(); - } + @FunctionParamter(value = "string", comment = "正则表达式") String regex) { + String log = FunctionUtils.getValueString(message, context, fieldName); - JSONObject jsonObject = parseLog(regex, fieldName, log, keyNames); + regex=FunctionUtils.getConstant(regex); + JSONObject jsonObject = parseLog(regex, fieldName, log); if (jsonObject == null) { context.breakExecute(); } @@ -54,11 +53,10 @@ public String paserByRegex(IMessage message, FunctionContext context, * 解析实例日志 * * @param regex 正则表达式 - * @param keyNames 正则表达式中的字段 * @param log 日志 * @return regex和解析的字段和内容的对应关系 */ - public static JSONObject parseLog(String regex, String fieldName, String log, String... keyNames) { + public static JSONObject parseLog(String regex, String fieldName, String log) { JSONObject result = new JSONObject(); Pattern pattern = Pattern.compile(regex); Matcher matcher = pattern.matcher(log); @@ -67,15 +65,7 @@ public static JSONObject parseLog(String regex, String fieldName, String log, St return null; } for (int i = 1; i <= matcher.groupCount(); i++) { - String name = null; - if ((i - 1) < keyNames.length) { - name = keyNames[i - 1]; - if (StringUtil.isEmpty(name)) { - name = fieldName + (i - 1); - } - } else { - name = fieldName + (i - 1); - } + String name = FunctionType.UDTF.getName()+i; result.put(name, matcher.group(i)); } return result; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/router/RouteFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/router/RouteFunction.java index fc9191d5..eb65960e 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/router/RouteFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/router/RouteFunction.java @@ -26,12 +26,12 @@ public class RouteFunction { @FunctionMethod(value = "route", comment = "做条件判断和路径选择") public String doRoute(IMessage message, FunctionContext context, String... routeLable) { - return message.getHeader().addRouteLable(routeLable); + return message.getHeader().addRouteLabel(routeLable); } @FunctionMethod(value = "filter", comment = "做条件判断和路径选择") public String doFilter(IMessage message, FunctionContext context, String... routeLable) { - return message.getHeader().addFilterLable(routeLable); + return message.getHeader().addFilterLabel(routeLable); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java index d7b6553d..d8d8cde3 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java @@ -17,8 +17,10 @@ package org.apache.rocketmq.streams.script.operator.impl; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -45,7 +47,13 @@ public class AggregationScript implements IStreamOperator> { private static final Log LOG = LogFactory.getLog(AggregationScript.class); - + private static Set supportQuickStoreModelFunctions=new HashSet(){{ + add("max"); + add("min"); + add("count"); + add("sum"); + add("avg"); + }}; private static Map aggregationEngineMap = new ConcurrentHashMap() {{ put("max", MaxAccumulator.class); put("min", MinAccumulator.class); @@ -113,6 +121,11 @@ public AggregationScript clone() { return theClone; } + + public boolean supportQuickStoreModel(){ + return supportQuickStoreModelFunctions.contains(this.functionName); + } + //region setter and getter public String getFunctionName() { @@ -196,6 +209,10 @@ public static IAccumulator getAggregationFunction(String functionName) { return null; } + + public static void registUDAF(String functionName,Class accumulator){ + aggregationEngineMap.put(functionName,accumulator); + } public Object getAccumulator() { return accumulator; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/IAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/IAccumulator.java index a533e002..b5c61442 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/IAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/IAccumulator.java @@ -16,10 +16,12 @@ */ package org.apache.rocketmq.streams.script.service; +import java.io.Serializable; + /** * UDAF的标准接口,所有的udaf都要实现这个接口。 Blink UDAF也是生成这个接口的实现类来完成转化的 */ -public interface IAccumulator { +public interface IAccumulator extends Serializable { String ACCUMULATOR_VALUE = "value"; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/SimpleUDAFScript.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/SimpleUDAFScript.java new file mode 100644 index 00000000..936df8f3 --- /dev/null +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/SimpleUDAFScript.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.rocketmq.streams.script.service.udf; + +import java.io.Serializable; + +public class SimpleUDAFScript extends UDAFScript { + + public SimpleUDAFScript(){ + this.accumulateMethodName = "accumulate"; + this.createAccumulatorMethodName = "createAccumulator"; + this.getValueMethodName = "getValue"; + this.retractMethodName = "retract"; + this.mergeMethodName = "merge"; + this.methodName = "eval"; + this.initMethodName = "open"; + this.initParameters = new Object[0]; + } + + @Override protected Object createMergeParamters(Iterable its) { + return its; + } +} diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDFScript.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDFScript.java index ff5e9945..c8f0e927 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDFScript.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDFScript.java @@ -76,7 +76,8 @@ public class UDFScript extends AbstractScript implements IScriptUDFInit { @Override protected boolean initConfigurable() { registFunctionSerivce(scriptComponent.getFunctionService()); - FunctionConfigure functionConfigure = scriptComponent.getFunctionService().getFunctionConfigure(createInitMethodName(), this.initParameters); + FunctionConfigure functionConfigure = + scriptComponent.getFunctionService().getFunctionConfigure(createInitMethodName(), this.initParameters); if (functionConfigure == null) { return true; } @@ -147,14 +148,23 @@ public void destroy() { * @return */ protected boolean initBeanClass(IFunctionService iFunctionService) { + + ClassLoader classLoader = this.getClass().getClassLoader(); + Class clazz; + try { + clazz = classLoader.loadClass(fullClassName); + instance = clazz.newInstance(); + return true; + } catch (Exception e) { + e.printStackTrace(); + } try { - ClassLoader classLoader = this.getClass().getClassLoader(); String jarUrl = getValue(); - if (StringUtil.isEmpty(jarUrl)) { - Class clazz = classLoader.loadClass(fullClassName); - instance = clazz.newInstance(); - return true; - } +// if (StringUtil.isEmpty(jarUrl)) { +// clazz = classLoader.loadClass(fullClassName); +// instance = clazz.newInstance(); +// return true; +// } URL url = null; if (isURL) { url = new URL(getValue()); @@ -178,7 +188,7 @@ protected boolean initBeanClass(IFunctionService iFunctionService) { URLClassLoader urlClassLoader = new URLClassLoader(urls, classLoader); classLoader = urlClassLoader; - Class clazz = classLoader.loadClass(fullClassName); + clazz = classLoader.loadClass(fullClassName); instance = clazz.newInstance(); } catch (Exception e) { diff --git a/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/FunctionTest.java b/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/FunctionTest.java index 59f8a23d..75a0d31b 100644 --- a/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/FunctionTest.java +++ b/rocketmq-streams-script/src/test/java/org/apache/rocketmq/streams/script/function/FunctionTest.java @@ -22,6 +22,7 @@ import java.util.List; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.utils.JsonableUtil; import org.apache.rocketmq.streams.script.ScriptComponent; import org.apache.rocketmq.streams.script.context.FunctionContext; import org.junit.Test; @@ -49,7 +50,7 @@ public void testDate() { message.put("from", "2019-07-14 00:00:00"); message.put("last", "2019-07-14 01:00:00"); message.put("event_type", "alert"); - String scriptValue = "now=now();nowhh=datefirst(now,'hh');from=dateAdd(last,'hh',-1);"; + String scriptValue = "now=now();x =concat(from,last)"; List list = ScriptComponent.getInstance().getService().executeScript(message, scriptValue); for (int i = 0; i < list.size(); i++) { assertTrue(list.get(i).getMessageBody().getString("from").equals("2019-07-14 00:00:00")); @@ -57,6 +58,22 @@ public void testDate() { } } + @Test + public void testJSON(){ + JSONObject jsonObject=new JSONObject(); + JSONObject person=new JSONObject(); + person.put("name","chris"); + person.put("age",18); + jsonObject.put("persion",person); + JSONArray jsonArray=new JSONArray(); + for(int i=0;i<3;i++){ + JSONObject jsonObject1=new JSONObject(); + jsonObject1.put("address","address"+i); + jsonArray.add(jsonObject1); + } + jsonObject.put("addresses",jsonArray); + System.out.println(JsonableUtil.formatJson(jsonObject)); + } /** * 在导表时常用的语句 diff --git a/rocketmq-streams-serviceloader/pom.xml b/rocketmq-streams-serviceloader/pom.xml index be52a041..a1cf354d 100755 --- a/rocketmq-streams-serviceloader/pom.xml +++ b/rocketmq-streams-serviceloader/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.rocketmq diff --git a/rocketmq-streams-state/pom.xml b/rocketmq-streams-state/pom.xml index 32aa6833..6e548a5f 100644 --- a/rocketmq-streams-state/pom.xml +++ b/rocketmq-streams-state/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.rocketmq diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksDBOperator.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksDBOperator.java index 87453f0e..a06ca581 100644 --- a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksDBOperator.java +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksDBOperator.java @@ -21,11 +21,22 @@ import org.apache.rocketmq.streams.common.utils.FileUtil; import org.apache.rocketmq.streams.common.utils.RuntimeUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.rocksdb.BlockBasedTableConfig; +import org.rocksdb.BloomFilter; +import org.rocksdb.Cache; +import org.rocksdb.CompactionStyle; +import org.rocksdb.CompressionType; +import org.rocksdb.Filter; +import org.rocksdb.LRUCache; import org.rocksdb.Options; +import org.rocksdb.RateLimiter; +import org.rocksdb.ReadOptions; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; +import org.rocksdb.Statistics; import org.rocksdb.TtlDB; import org.rocksdb.WriteOptions; +import org.rocksdb.util.SizeUnit; public class RocksDBOperator { @@ -61,9 +72,33 @@ public RocksDBOperator(String rocksdbFilePath) { dir.delete(); } dir.mkdirs(); + // final Filter bloomFilter = new BloomFilter(10); +// final ReadOptions readOptions = new ReadOptions().setFillCache(false); +// final Statistics stats = new Statistics(); +// final RateLimiter rateLimiter = new RateLimiter(10000000, 10000, 10); +// +// options.setCreateIfMissing(true) +// // .setStatistics(stats) +// .setWriteBufferSize(64*1024 * SizeUnit.KB) +// .setMaxWriteBufferNumber(3); +// .setMaxBackgroundJobs(10) +// .setCompressionType(CompressionType.SNAPPY_COMPRESSION) +// .setCompactionStyle(CompactionStyle.UNIVERSAL); +// +// final BlockBasedTableConfig table_options = new BlockBasedTableConfig(); +// Cache cache = new LRUCache(10 * 1024, 6); +// table_options.setBlockCache(cache) +// .setFilterPolicy(bloomFilter); +// .setBlockSizeDeviation(5) +// .setBlockRestartInterval(10) +// .setCacheIndexAndFilterBlocks(true); +// // .setBlockCacheCompressed(new LRUCache(64 * 1000, 10)); +// options.setTableFormatConfig(table_options); +// +// options.setRateLimiter(rateLimiter); final TtlDB db = TtlDB.open(options, rocksdbFilePath, 10800, false); RocksDBOperator.rocksDB = db; - writeOptions.setSync(true); + writeOptions.setSync(false); } catch (RocksDBException e) { throw new RuntimeException("create rocksdb error " + e.getMessage()); } diff --git a/rocketmq-streams-transport-minio/pom.xml b/rocketmq-streams-transport-minio/pom.xml index 721e2032..3637d333 100755 --- a/rocketmq-streams-transport-minio/pom.xml +++ b/rocketmq-streams-transport-minio/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.rocketmq diff --git a/rocketmq-streams-window/pom.xml b/rocketmq-streams-window/pom.xml index 86d276bc..8624079d 100755 --- a/rocketmq-streams-window/pom.xml +++ b/rocketmq-streams-window/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.rocketmq @@ -39,10 +41,13 @@ org.rocksdb rocksdbjni - org.apache.rocketmq rocketmq-tools + + org.apache.rocketmq + rocketmq-streams-commons + diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/minibatch/ShuffleMessageCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/minibatch/ShuffleMessageCache.java new file mode 100644 index 00000000..aa394291 --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/minibatch/ShuffleMessageCache.java @@ -0,0 +1,187 @@ +/* + * 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.rocketmq.streams.window.minibatch; + +import com.alibaba.fastjson.JSONObject; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack; +import org.apache.rocketmq.streams.common.channel.sinkcache.impl.MessageCache; +import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.context.MessageHeader; +import org.apache.rocketmq.streams.common.topology.shuffle.IShuffleKeyGenerator; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.script.operator.impl.AggregationScript; +import org.apache.rocketmq.streams.window.model.WindowCache; +import org.apache.rocketmq.streams.window.model.WindowInstance; +import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; +import org.apache.rocketmq.streams.window.operator.AbstractWindow; +import org.apache.rocketmq.streams.window.operator.impl.WindowOperator; +import org.apache.rocketmq.streams.window.state.impl.WindowValue; +import org.apache.rocketmq.streams.window.util.ShuffleUtil; + +public class ShuffleMessageCache extends MessageCache> { + + protected Map groupBy2WindowValue=new HashMap<>(); + + + protected transient IShuffleKeyGenerator shuffleKeyGenerator; + protected transient AbstractShuffleWindow window; + public ShuffleMessageCache( + IMessageFlushCallBack> flushCallBack) { + super(flushCallBack); + } + protected class MiniBatch{ + protected WindowValue windowValue; + protected IMessage message; + + public MiniBatch(){ + windowValue=new WindowValue(); + } + public IMessage calculate(AbstractWindow window, IMessage msg,String groupByValue) { + windowValue.calculate(window,msg); + JSONObject mergeMsg=createMsg(groupByValue,windowValue,msg.getHeader(),msg.getMessageBody().getJSONObject(WindowCache.ORIGIN_MESSAGE_HEADER)); + if(window.getTimeFieldName()!=null){ + mergeMsg.put(window.getTimeFieldName(),msg.getMessageBody().getString(window.getTimeFieldName())); + } + if(msg.getMessageBody().get(WindowInstance.class.getSimpleName())!=null){ + mergeMsg.put(WindowInstance.class.getSimpleName(),msg.getMessageBody().get(WindowInstance.class.getSimpleName())); + } + if(msg.getMessageBody().get(AbstractWindow.class.getSimpleName())!=null){ + mergeMsg.put(AbstractWindow.class.getSimpleName(),msg.getMessageBody().get(AbstractWindow.class.getSimpleName())); + } + + + + if(this.message==null){ + this.message=new Message(mergeMsg); + return message; + }else { + this.message.getMessageBody().putAll(mergeMsg); + } + + + return null; + } + + } + + + @Override public synchronized int addCache(Pair pair) { + boolean openMiniBatch=isOpenMiniBatch(); + ISplit split=pair.getLeft(); + IMessage message=pair.getRight(); + if(openMiniBatch){ + String groupByValue = shuffleKeyGenerator.generateShuffleKey(message); + if (StringUtil.isEmpty(groupByValue)) { + groupByValue = ""; + } + List windowInstances = (List) message.getMessageBody().get(WindowInstance.class.getSimpleName()); + if(windowInstances==null){ + windowInstances=this.window.queryOrCreateWindowInstanceOnly(message,split.getQueueId()); + } + for(WindowInstance windowInstance:windowInstances){ + String key= MapKeyUtil.createKey(windowInstance.createWindowInstanceId(),groupByValue); + MiniBatch miniBatch = groupBy2WindowValue.get(key); + if(miniBatch==null) { + miniBatch = new MiniBatch(); + groupBy2WindowValue.put(key, miniBatch); + + } + IMessage newMergeMessage=miniBatch.calculate(this.window,message,groupByValue); + if(newMergeMessage!=null){ + pair.setValue(newMergeMessage); + return super.addCache(pair); + } + } + }else { + return super.addCache(pair); + } + + return 0; + } + + protected boolean isOpenMiniBatch() { + if(!WindowOperator.class.isInstance(window)){ + return false; + } + if(window.getGroupByFieldName()==null&&WindowOperator.class.getSimpleName().equals(window.getClass().getSimpleName())){ + return true; + } + boolean isOpenMiniBatch= ComponentCreator.getPropertyBooleanValue(ConfigureFileKey.WINDOW_MINIBATCH_SWITCH); + return isOpenMiniBatch; + } + + protected transient AtomicLong SUM=new AtomicLong(0); + protected JSONObject createMsg(String shuffleKey,WindowValue windowValue, MessageHeader messageHeader,JSONObject msgHeader) { + + JSONObject message = new JSONObject(); + long start=System.currentTimeMillis(); + message.put(WindowValue.class.getName(), windowValue); + // long sum=SUM.addAndGet(System.currentTimeMillis()-start); + // System.out.println("create msg "+sum); + message.put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY,AggregationScript.INNER_AGGREGATION_COMPUTE_MULTI); + IMessage windowValueMsg=new Message(message); + windowValueMsg.setHeader(messageHeader); + ShuffleUtil.createShuffleMsg(windowValueMsg,shuffleKey,msgHeader); + + if (JSONObject.class.isInstance(windowValue.getcomputedResult())) { + message.putAll(windowValue.getcomputedResult()); + } else { + Iterator> it = windowValue.iteratorComputedColumnResult(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + message.put(entry.getKey(), entry.getValue()); + } + } + + return message; + + } + + + @Override public synchronized int flush() { + this.groupBy2WindowValue=new HashMap<>(); + return super.flush(); + } + + public IShuffleKeyGenerator getShuffleKeyGenerator() { + return shuffleKeyGenerator; + } + + public void setShuffleKeyGenerator( + IShuffleKeyGenerator shuffleKeyGenerator) { + this.shuffleKeyGenerator = shuffleKeyGenerator; + } + + public AbstractShuffleWindow getWindow() { + return window; + } + + public void setWindow(AbstractShuffleWindow window) { + this.window = window; + } +} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java index c749bd84..4062b1f5 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java @@ -116,6 +116,10 @@ public WindowInstance copy() { return windowInstance; } + public String createWindowInstanceId() { + return MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime); + } + public String createWindowInstanceTriggerId() { return MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime, fireTime); } @@ -153,7 +157,6 @@ public static Long getOccurTime(AbstractWindow window, IMessage message) { } return occurTime; } - /** * 查询或者创建Window的实例,滑动窗口有可能返回多个,滚动窗口返回一个 * @@ -163,6 +166,17 @@ public static Long getOccurTime(AbstractWindow window, IMessage message) { * @Param isWindowInstance2DB 如果是秒级窗口,可能windowinstacne不必存表,只在内存保存,可以通过这个标志设置 */ public static List getOrCreateWindowInstance(AbstractWindow window, Long occurTime, int timeUnitAdjust, String queueId) { + return getOrCreateWindowInstance(window,occurTime,timeUnitAdjust,queueId,false); + } + /** + * 查询或者创建Window的实例,滑动窗口有可能返回多个,滚动窗口返回一个 + * + * @param window + * @param occurTime + * @return + * @Param isWindowInstance2DB 如果是秒级窗口,可能windowinstacne不必存表,只在内存保存,可以通过这个标志设置 + */ + public static List getOrCreateWindowInstance(AbstractWindow window, Long occurTime, int timeUnitAdjust, String queueId, boolean isCreateOnly) { int windowSlideInterval = window.getSlideInterval(); int windowSizeInterval = window.getSizeInterval(); if (windowSlideInterval == 0) { @@ -183,6 +197,9 @@ public static List getOrCreateWindowInstance(AbstractWindow wind for (Date begin : windowBeginTimeList) { Date end = DateUtil.addDate(TimeUnit.SECONDS, begin, windowSizeInterval * timeUnitAdjust); Date fire = null; + /** + * if 这段代码可以忽略,全部用模式0和emit组合实现 + */ if (window.getFireMode() != 0) { //非正常触发模式 if (maxEventTime == null || maxEventTime - end.getTime() < 0) { @@ -204,6 +221,7 @@ public static List getOrCreateWindowInstance(AbstractWindow wind } /** * mode 2 clear window instance in first create window instance + * 已经不再使用,这段代码可以忽略 */ if (window.getFireMode() == 2 && fire.getTime() == end.getTime() && waterMarkMinute > 0) { Date clearWindowInstanceFireTime = DateUtil.addDate(TimeUnit.SECONDS, end, waterMarkMinute * timeUnitAdjust); @@ -249,7 +267,7 @@ public static List getOrCreateWindowInstance(AbstractWindow wind //todo 这里针对lost的都创建一次 lostInstanceList = WindowInstance.createWindowInstances(window, lostWindowTimeList, lostFireList, queueId); instanceList.addAll(lostInstanceList); - if (CollectionUtil.isNotEmpty(lostInstanceList)) { + if (CollectionUtil.isNotEmpty(lostInstanceList)&&!isCreateOnly) { for (WindowInstance windowInstance : instanceList) { List emitInstances = createEmitWindowInstance(window, windowInstance); if (emitInstances != null && emitInstances.size() > 0) { diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java index eb513c4b..817401f1 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java @@ -17,14 +17,20 @@ package org.apache.rocketmq.streams.window.operator; import com.alibaba.fastjson.JSONObject; + +import javafx.util.Pair; import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.channel.sink.ISink; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; +import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; +import org.apache.rocketmq.streams.common.configurable.IConfigurableService; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.common.context.MessageHeader; +import org.apache.rocketmq.streams.common.functions.MapFunction; import org.apache.rocketmq.streams.common.topology.ChainStage; import org.apache.rocketmq.streams.common.topology.SectionPipeline; import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder; @@ -58,7 +64,7 @@ /** * window definition in the pipeline, created by user's configure in WindowChainStage */ -public abstract class AbstractWindow extends BasedConfigurable implements IWindow, IStageBuilder { +public abstract class AbstractWindow extends BasedConfigurable implements IAfterConfigurableRefreshListener, IWindow, IStageBuilder { protected static final Log LOG = LogFactory.getLog(AbstractWindow.class); @@ -129,6 +135,8 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo protected Long emitBeforeValue;//output frequency before window fire protected Long emitAfterValue;// output frequency after window fire protected Long maxDelay = 60 * 60L;//when emitAfterValue>0, window last delay time after window fired + + protected String contextMsgSinkName;//上下文消息保存sink /** * 是否支持过期数据的计算 过期:当前时间大于数据所在窗口的触发时间 */ @@ -137,12 +145,13 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo protected boolean isLocalStorageOnly = true;//是否只用本地存储,可以提高性能,但不保证可靠性 protected String reduceSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码 protected transient IReducer reducer; - protected transient Long maxPartitionNum = 100000000L; + protected String mapFunctionSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码 + protected transient MapFunction> mapFunction; /** * the computed column and it's process of computing */ - private transient Map> columnExecuteMap = new HashMap<>(16); + protected transient Map> columnExecuteMap = new HashMap<>(16); /** * used in last part to filter and transfer field in case data lost during firing @@ -169,6 +178,7 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo protected transient IStorage storage; protected transient WindowTrigger windowFireSource; protected transient EventTimeManager eventTimeManager; + protected transient ISink contextMsgSink; public AbstractWindow() { setType(IWindow.TYPE); @@ -199,6 +209,10 @@ protected String generateShuffleKey(IMessage message) { byte[] bytes = Base64Utils.decode(this.reduceSerializeValue); reducer = InstantiationUtil.deserializeObject(bytes); } + if (StringUtil.isNotEmpty(this.mapFunctionSerializeValue)) { + byte[] bytes = Base64Utils.decode(this.mapFunctionSerializeValue); + this.mapFunction = InstantiationUtil.deserializeObject(bytes); + } eventTimeManager = new EventTimeManager(); @@ -306,9 +320,10 @@ public long incrementAndGetSplitNumber(WindowInstance instance, String shuffleId * @param message * @return */ - protected String generateShuffleKey(IMessage message) { + @Override + public String generateShuffleKey(IMessage message) { if (StringUtil.isEmpty(groupByFieldName)) { - return null; + return "globle_window"; } JSONObject msg = message.getMessageBody(); @@ -422,6 +437,10 @@ public List queryOrCreateWindowInstance(IMessage message, String return WindowInstance.getOrCreateWindowInstance(this, WindowInstance.getOccurTime(this, message), timeUnitAdjust, queueId); } + public List queryOrCreateWindowInstanceOnly(IMessage message, String queueId) { + return WindowInstance.getOrCreateWindowInstance(this, WindowInstance.getOccurTime(this, message), timeUnitAdjust, queueId,true); + } + public WindowInstance registerWindowInstance(WindowInstance windowInstance) { return registerWindowInstance(windowInstance.createWindowInstanceTriggerId(), windowInstance); } @@ -528,6 +547,10 @@ public void sendFireMessage(List windowValueList, String queueId) { } } + @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { + this.contextMsgSink=configurableService.queryConfigurable(ISink.TYPE,contextMsgSinkName); + } + @Override public ChainStage createStageChain(PipelineBuilder pipelineBuilder) { pipelineBuilder.addConfigurables(this); @@ -781,4 +804,44 @@ public void setMaxPartitionNum(Long maxPartitionNum) { } public abstract boolean supportBatchMsgFinish(); + + public ISink getContextMsgSink() { + return contextMsgSink; + } + + public String getContextMsgSinkName() { + return contextMsgSinkName; + } + + public void setContextMsgSinkName(String contextMsgSinkName) { + this.contextMsgSinkName = contextMsgSinkName; + } + + public String getMapFunctionSerializeValue() { + return mapFunctionSerializeValue; + } + + public void setMapFunctionSerializeValue(String mapFunctionSerializeValue) { + this.mapFunctionSerializeValue = mapFunctionSerializeValue; + } + + public void saveMsgContext(String queueId,WindowInstance windowInstance, List messages) { + if(this.mapFunction!=null&&this.contextMsgSink!=null){ + if(messages!=null){ + for(IMessage message:messages){ + JSONObject msg=message.getMessageBody(); + try { + msg=this.mapFunction.map(new Pair(windowInstance,msg)); + Message copyMsg=new Message(msg); + copyMsg.getHeader().setQueueId(queueId); + copyMsg.getHeader().setOffset(message.getHeader().getOffset()); + this.contextMsgSink.batchAdd(copyMsg); + } catch (Exception e) { + throw new RuntimeException("save window context msg error ",e); + } + } + this.contextMsgSink.flush(); + } + } + } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java index 65ac261b..14e1ffd3 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java @@ -524,4 +524,12 @@ public long incrementAndGetSplitNumber(WindowInstance instance, String shuffleId } return numer; } -} + + public int getSessionTimeOut() { + return sessionTimeOut; + } + + public void setSessionTimeOut(int sessionTimeOut) { + this.sessionTimeOut = sessionTimeOut; + } +} \ No newline at end of file diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java index 65f326f6..de6dce22 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.streams.window.operator.impl; + import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.MessageOffset; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; @@ -29,7 +30,11 @@ import org.apache.rocketmq.streams.window.storage.RocksdbIterator; import org.apache.rocketmq.streams.window.storage.WindowType; -import java.util.*; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.stream.Collectors; @@ -88,6 +93,7 @@ private int sendBatch(List windowValues, String queueId, int fireCo @Override public void shuffleCalculate(List messages, WindowInstance instance, String queueId) { + Long startTime=System.currentTimeMillis(); DebugWriter.getDebugWriter(getConfigureName()).writeShuffleCalcultateReceveMessage(instance, messages, queueId); List sortKeys = new ArrayList<>(); diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java index 5b5356b3..323ea1dc 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java @@ -59,7 +59,7 @@ public class JoinWindow extends AbstractShuffleWindow { protected String joinType;//join类型,值为INNER,LEFT protected String expression;//条件表达式。在存在非等值比较时使用 - + protected String rightDependentTableName; @Override protected int doFireWindowInstance(WindowInstance instance) { @@ -358,7 +358,7 @@ public static String generateKey(JSONObject messageBody, String joinLabel, List< } @Override - protected String generateShuffleKey(IMessage message) { + public String generateShuffleKey(IMessage message) { String routeLabel = message.getHeader().getMsgRouteFromLable(); String messageKey = generateKey(message.getMessageBody(), routeLabel, leftJoinFieldNames, rightJoinFieldNames); return messageKey; @@ -556,4 +556,12 @@ public String getExpression() { public void setExpression(String expression) { this.expression = expression; } + + public String getRightDependentTableName() { + return rightDependentTableName; + } + + public void setRightDependentTableName(String rightDependentTableName) { + this.rightDependentTableName = rightDependentTableName; + } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java index 6c4659b3..0a3845fd 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java @@ -20,11 +20,11 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder; @@ -46,6 +46,7 @@ import org.apache.rocketmq.streams.common.utils.ReflectUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent; +import org.apache.rocketmq.streams.window.operator.AbstractWindow; public abstract class AbstractSystemChannel implements IConfigurableIdentification, ISystemMessageProcessor, IStreamOperator { @@ -54,21 +55,45 @@ public abstract class AbstractSystemChannel implements IConfigurableIdentificati protected static final String CHANNEL_PROPERTY_KEY_PREFIX = "CHANNEL_PROPERTY_KEY_PREFIX"; protected static final String CHANNEL_TYPE = "CHANNEL_TYPE"; - protected ISource consumer; + protected ISource consumer; protected AbstractSupportShuffleSink producer; protected Map channelConfig = new HashMap<>(); protected boolean hasCreateShuffleChannel = false; + protected transient AtomicBoolean hasStart = new AtomicBoolean(false); public void startChannel() { if (consumer == null) { return; } - final AbstractSystemChannel channel = this; - consumer.start(this); + if (hasStart.compareAndSet(false, true)) { + consumer.start(this); + } + } + /** - * 如果用户未配置shuffle channel,根据pipline数据源动态创建 + * init shuffle channel + */ + public void init(AbstractWindow window) { + this.consumer = createSource(window.getNameSpace(), window.getConfigureName()); + this.producer = createSink(window.getNameSpace(), window.getConfigureName()); + if (this.consumer == null || this.producer == null) { + autoCreateShuffleChannel(window.getFireReceiver().getPipeline()); + } + if (this.consumer == null) { + return; + } + if (this.consumer instanceof AbstractSource) { + ((AbstractSource) this.consumer).setJsonData(true); + } + if(producer!=null){ + this.producer.init(); + } + + } + /** + * 如果用户未配置shuffle channel,根据pipeline数据源动态创建 * * @param pipeline */ @@ -76,23 +101,23 @@ public void autoCreateShuffleChannel(ChainPipeline pipeline) { if (!hasCreateShuffleChannel) { synchronized (this) { if (!hasCreateShuffleChannel) { - ISource piplineSource = pipeline.getSource(); + ISource pipelineSource = pipeline.getSource(); ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(IChannelBuilder.class.getName(), ServiceLoaderComponent.class); - IChannelBuilder builder = (IChannelBuilder) serviceLoaderComponent.loadService(piplineSource.getClass().getSimpleName()); + IChannelBuilder builder = (IChannelBuilder) serviceLoaderComponent.loadService(pipelineSource.getClass().getSimpleName()); if (builder == null) { - throw new RuntimeException("can not create shuffle channel, not find channel builder " + piplineSource.toJson()); + throw new RuntimeException("can not create shuffle channel, not find channel builder " + pipelineSource.toJson()); } if (!(builder instanceof IShuffleChannelBuilder)) { - throw new RuntimeException("can not create shuffle channel, builder not impl IShuffleChannelBuilder " + piplineSource.toJson()); + throw new RuntimeException("can not create shuffle channel, builder not impl IShuffleChannelBuilder " + pipelineSource.toJson()); } IShuffleChannelBuilder shuffleChannelBuilder = (IShuffleChannelBuilder) builder; - ISink sink = shuffleChannelBuilder.createBySource(piplineSource); + ISink sink = shuffleChannelBuilder.createBySource(pipelineSource); sink.init(); if (!(sink instanceof MemoryChannel) && !(sink instanceof AbstractSupportShuffleSink)) { - throw new RuntimeException("can not create shuffle channel, sink not extends AbstractSupportShuffleSink " + piplineSource.toJson()); + throw new RuntimeException("can not create shuffle channel, sink not extends AbstractSupportShuffleSink " + pipelineSource.toJson()); } - ISource source = null; + ISource source = null; if (sink instanceof MemoryChannel) { MemoryCache memoryCache = new MemoryCache(); memoryCache.setNameSpace(createShuffleChannelNameSpace(pipeline)); @@ -134,7 +159,7 @@ public void autoCreateShuffleChannel(ChainPipeline pipeline) { shuffleSink.setHasInit(false); shuffleSink.init();//在这里完成shuffle channel的创建 if (source == null) { - source = shuffleChannelBuilder.copy(piplineSource); + source = shuffleChannelBuilder.copy(pipelineSource); } //修改和window有关的属性,如groupname,tags @@ -154,7 +179,7 @@ public void autoCreateShuffleChannel(ChainPipeline pipeline) { if (shuffleTopic != null && topicFiledName != null) { ReflectUtil.setBeanFieldValue(source, topicFiledName, shuffleTopic); } - if (AbstractSource.class.isInstance(source)) { + if (source instanceof AbstractSource) { AbstractSource abstractSource = (AbstractSource) source; abstractSource.setHasInit(false); } @@ -205,15 +230,14 @@ protected Map getChannelConfig() { * * @return */ - protected ISource createSource(String namespace, String name) { - + protected ISource createSource(String namespace, String name) { IChannelBuilder builder = createBuilder(); if (builder == null) { return null; } Properties properties = createChannelProperties(namespace); - ISource source = builder.createSource(namespace, name, properties, null); - if (MemorySource.class.isInstance(source)) { + ISource source = builder.createSource(namespace, name, properties, null); + if (source instanceof MemorySource) { MemorySource memorySource = (MemorySource) source; MemoryCache memoryCache = new MemoryCache(); memorySource.setMemoryCache(memoryCache); @@ -236,13 +260,13 @@ protected AbstractSupportShuffleSink createSink(String namespace, String name) { } Properties properties = createChannelProperties(namespace); - ISink sink = builder.createSink(namespace, name, properties, null); - if (!AbstractSupportShuffleSink.class.isInstance(sink)) { + ISink sink = builder.createSink(namespace, name, properties, null); + if (!(sink instanceof AbstractSupportShuffleSink)) { throw new RuntimeException("can not support shuffle " + sink.toJson()); } - if (MemorySink.class.isInstance(sink)) { + if (sink instanceof MemorySink) { MemorySink memorySink = (MemorySink) sink; - if (!MemorySource.class.isInstance(this.consumer)) { + if (!(this.consumer instanceof MemorySource)) { throw new RuntimeException("shuffle cosumer need memory, real is " + this.consumer); } MemorySource memorySource = (MemorySource) this.consumer; @@ -251,7 +275,6 @@ protected AbstractSupportShuffleSink createSink(String namespace, String name) { } sink.init(); - return (AbstractSupportShuffleSink) sink; } @@ -266,20 +289,17 @@ protected IChannelBuilder createBuilder() { return null; } ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(IChannelBuilder.class.getName(), ServiceLoaderComponent.class); - IChannelBuilder builder = (IChannelBuilder) serviceLoaderComponent.loadService(type); - return builder; + return (IChannelBuilder) serviceLoaderComponent.loadService(type); } /** * 根据属性文件配置 * - * @return + * @return 资源文件 */ protected Properties createChannelProperties(String namespace) { Properties properties = new Properties(); - Iterator> it = ComponentCreator.getProperties().entrySet().iterator(); - while (it.hasNext()) { - Map.Entry entry = it.next(); + for (Map.Entry entry : ComponentCreator.getProperties().entrySet()) { String key = (String) entry.getKey(); String value = (String) entry.getValue(); if (key.startsWith(getChannelConfig().get(CHANNEL_PROPERTY_KEY_PREFIX))) { @@ -292,11 +312,9 @@ protected Properties createChannelProperties(String namespace) { properties.put(channelKey, value); } } - } - } - Set mutilPropertySet = new HashSet<>(); + Set multiPropertySet = new HashSet<>(); String dynamicProperty = properties.getProperty("dynamic.property"); if (dynamicProperty != null) { @@ -305,28 +323,28 @@ protected Properties createChannelProperties(String namespace) { for (String properyKey : mutilPropertys) { properties.put(properyKey, dynamicPropertyValue); - mutilPropertySet.add(properyKey); + multiPropertySet.add(properyKey); } } - putDynamicPropertyValue(mutilPropertySet, properties); + putDynamicPropertyValue(multiPropertySet, properties); return properties; } /** * 如果需要额外的动态属性,可以在子类添加 * - * @param dynamiPropertySet + * @param dynamicPropertySet 资源集 */ - protected void putDynamicPropertyValue(Set dynamiPropertySet, Properties properties) { + protected void putDynamicPropertyValue(Set dynamicPropertySet, Properties properties) { } - public ISource getConsumer() { + public ISource getConsumer() { return consumer; } - public ISink getProducer() { + public ISink getProducer() { return producer; } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java index 5139f707..7b4d8b4f 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java @@ -23,8 +23,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Future; - import org.apache.commons.lang3.tuple.Pair; +import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.context.MessageOffset; import org.apache.rocketmq.streams.window.debug.DebugWriter; @@ -36,7 +36,7 @@ /** * save receiver messages into cachefilter when checkpoint/autoflush/flush, process cachefilter message */ -public class ShuffleCache extends WindowCache { +public class ShuffleCache extends AbstractSink { protected AbstractShuffleWindow window; private HashMap hasLoad = new HashMap<>(); @@ -77,8 +77,9 @@ protected boolean batchInsert(List messageList) { //保存处理进度 saveSplitProgress(queueId, messages); + window.saveMsgContext(queueId,windowInstance,messages); } - return true; + return true; } private void stateMustLoad(String queueId) { @@ -125,7 +126,7 @@ protected void saveSplitProgress(String shuffleId, List messages) { Map queueId2OrigOffset = new HashMap<>(); Boolean isLong = false; for (IMessage message : messages) { - isLong = message.getMessageBody().getBoolean(ORIGIN_QUEUE_IS_LONG); + isLong = message.getMessageBody().getBoolean(WindowCache.ORIGIN_QUEUE_IS_LONG); String oriQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID); String oriOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET); queueId2OrigOffset.put(oriQueueId, oriOffset); @@ -143,11 +144,6 @@ protected void saveSplitProgress(String shuffleId, List messages) { } - @Override - protected String generateShuffleKey(IMessage message) { - return null; - } - /** * 根据message,把message分组到不同的group,分别处理 * diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java index f9bd3413..a5c150a1 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java @@ -19,6 +19,7 @@ import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.MutablePair; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage; @@ -27,9 +28,12 @@ import org.apache.rocketmq.streams.common.channel.source.systemmsg.NewSplitMessage; import org.apache.rocketmq.streams.common.channel.source.systemmsg.RemoveSplitMessage; import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.checkpoint.CheckPointManager; import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage; import org.apache.rocketmq.streams.common.checkpoint.CheckPointState; +import org.apache.rocketmq.streams.common.checkpoint.SourceState; import org.apache.rocketmq.streams.common.component.ComponentCreator; +import org.apache.rocketmq.streams.common.configurable.IConfigurableIdentification; import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; @@ -71,16 +75,21 @@ public class ShuffleChannel extends AbstractSystemChannel { protected static final String SHUFFLE_QUEUE_ID = "SHUFFLE_QUEUE_ID"; public static final String SHUFFLE_OFFSET = "SHUFFLE_OFFSET"; protected static final String SHUFFLE_MESSAGES = "SHUFFLE_MESSAGES"; - protected String MSG_OWNER = "MSG_OWNER";//消息所属的window + /** + * 消息所属的window + */ + protected String MSG_OWNER = "MSG_OWNER"; private static final String SHUFFLE_TRACE_ID = "SHUFFLE_TRACE_ID"; protected ShuffleCache shuffleCache; - protected Map queueMap = new ConcurrentHashMap<>(); - protected List queueList;//所有的分片 + protected Map> queueMap = new ConcurrentHashMap<>(); + /** + * 所有的分片 + */ + protected List> queueList; - // protected NotifyChannel notfiyChannel;//负责做分片的通知管理 protected AbstractShuffleWindow window; protected transient boolean isWindowTest = false; @@ -109,32 +118,19 @@ public void startChannel() { if (hasStart.compareAndSet(false, true)) { super.startChannel(); } - } /** * init shuffle channel */ public void init() { - this.consumer = createSource(window.getNameSpace(), window.getConfigureName()); - - this.producer = createSink(window.getNameSpace(), window.getConfigureName()); - if (this.consumer == null || this.producer == null) { - autoCreateShuffleChannel(window.getFireReceiver().getPipeline()); - } - if (this.consumer == null) { - return; - } - if (this.consumer instanceof AbstractSource) { - ((AbstractSource) this.consumer).setJsonData(true); - } + init(this.window); if (producer != null && (queueList == null || queueList.size() == 0)) { queueList = producer.getSplitList(); - Map tmp = new ConcurrentHashMap<>(); - for (ISplit queue : queueList) { + Map> tmp = new ConcurrentHashMap<>(); + for (ISplit queue : queueList) { tmp.put(queue.getQueueId(), queue); } - this.queueMap = tmp; } // isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest"); @@ -179,7 +175,6 @@ public Object doMessage(IMessage oriMessage, AbstractContext context) { if (!StringUtil.isEmpty(traceId)) { TraceUtil.debug(traceId, "shuffle message in", "received message size:" + messages.size()); } - for (Object obj : messages) { IMessage message = new Message((JSONObject) obj); message.getHeader().setQueueId(queueId); @@ -306,12 +301,12 @@ protected Map getChannelConfig() { } @Override - protected void putDynamicPropertyValue(Set dynamiPropertySet, Properties properties) { + protected void putDynamicPropertyValue(Set dynamicPropertySet, Properties properties) { String groupName = "groupName"; - if (!dynamiPropertySet.contains(groupName)) { + if (!dynamicPropertySet.contains(groupName)) { properties.put(groupName, getDynamicPropertyValue()); } - if (!dynamiPropertySet.contains("tags")) { + if (!dynamicPropertySet.contains("tags")) { properties.put("tags", getDynamicPropertyValue()); } } @@ -468,7 +463,7 @@ protected int getShuffleSplitCount(AbstractSupportShuffleSink shuffleSink) { } - public List getQueueList() { + public List> getQueueList() { return queueList; } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java index c8534541..7d7285c0 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java @@ -27,6 +27,8 @@ import java.util.Map; import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -34,11 +36,13 @@ import org.apache.rocketmq.streams.common.context.Message; import org.apache.rocketmq.streams.common.datatype.DataType; import org.apache.rocketmq.streams.common.datatype.NotSupportDataType; +import org.apache.rocketmq.streams.common.interfaces.ISerialize; import org.apache.rocketmq.streams.common.interfaces.IStreamOperator; import org.apache.rocketmq.streams.common.utils.Base64Utils; import org.apache.rocketmq.streams.common.utils.DataTypeUtil; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.common.utils.ReflectUtil; +import org.apache.rocketmq.streams.common.utils.SerializeUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.common.utils.TraceUtil; import org.apache.rocketmq.streams.script.context.FunctionContext; @@ -49,8 +53,9 @@ import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractWindow; import org.apache.rocketmq.streams.window.state.WindowBaseValue; +import org.nustaq.serialization.FSTConfiguration; -public class WindowValue extends WindowBaseValue implements Serializable { +public class WindowValue extends WindowBaseValue implements Serializable ,ISerialize{ private static final long serialVersionUID = 1083444850264401338L; @@ -64,7 +69,7 @@ public class WindowValue extends WindowBaseValue implements Serializable { /** * split id和max offset的映射关系 */ - protected ConcurrentHashMap maxOffset = new ConcurrentHashMap<>(16); + protected Map maxOffset = new ConcurrentHashMap<>(16); /** * the result of aggregation column @@ -259,9 +264,9 @@ public boolean calculate(AbstractWindow window, IMessage message) { return true; } calFunctionColumn(window, message); - calProjectColumn(window, message); + } catch (Exception e) { - LOG.error("failed in calculating the message", e); + throw new RuntimeException("failed in window value calculating",e); } //there is no need writing back to message @@ -269,7 +274,26 @@ public boolean calculate(AbstractWindow window, IMessage message) { return true; } + protected static AtomicInteger SUM=new AtomicInteger(0); + protected void calFunctionColumn(AbstractWindow window, IMessage message) { + String introduction = (String)message.getMessageBody().getOrDefault(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY, ""); + boolean isMultiAccumulate = AggregationScript.INNER_AGGREGATION_COMPUTE_MULTI.equals(introduction); + if(isMultiAccumulate){ + WindowValue windowValue=message.getMessageBody().getObject(WindowValue.class.getName(),WindowValue.class); + try { + // windowValue= SerializeUtil.deserialize(windowValueJson,WindowValue.class); + }catch (Exception e){ + throw new RuntimeException("window value deserializeObject error",e); + } + List windowValues=new ArrayList<>(); + windowValues.add(this); + windowValues.add(windowValue); + WindowValue mergerWindowValue= WindowValue.mergeWindowValue(window,windowValues); + this.computedColumnResult.putAll(mergerWindowValue.computedColumnResult); + this.aggColumnResult.putAll(mergerWindowValue.aggColumnResult); + return; + } for (Entry> entry : window.getColumnExecuteMap().entrySet()) { String computedColumn = entry.getKey(); List fifoQueue = entry.getValue(); @@ -289,8 +313,10 @@ protected void calFunctionColumn(AbstractWindow window, IMessage message) { aggColumnResult.put(executorName, accumulator); } windowAccScript.setAccumulator(accumulator); - message.getMessageBody().put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY, - AggregationScript.INNER_AGGREGATION_COMPUTE_SINGLE); + if(!isMultiAccumulate){ + message.getMessageBody().put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY, + AggregationScript.INNER_AGGREGATION_COMPUTE_SINGLE); + } FunctionContext context = new FunctionContext(message); windowAccScript.doMessage(message, context); } else if (executor instanceof FunctionScript) { @@ -301,6 +327,7 @@ protected void calFunctionColumn(AbstractWindow window, IMessage message) { // computedColumnResult.put(computedColumn, message.getMessageBody().get(computedColumn)); } + calProjectColumn(window, message); } protected void calProjectColumn(AbstractWindow window, IMessage message) { @@ -348,10 +375,22 @@ public static WindowValue mergeWindowValue(AbstractWindow window, List windowValue.getAccumulatorByColumn(column)).collect( + windowValue -> { + Object accumulator = null; + if (windowValue.aggColumnResult.containsKey(column)) { + accumulator = windowValue.aggColumnResult.get(column); + } else { + IAccumulator director = AggregationScript.getAggregationFunction( + operator.getFunctionName()); + accumulator = director.createAccumulator(); + windowValue.aggColumnResult.put(column, accumulator); + } + return accumulator; + }).collect( Collectors.toList()); operator.setAccumulator(operator.getDirector().createAccumulator()); operator.setAccumulators(actors); + lastWindowValue.aggColumnResult.put(column,operator.getAccumulator()); operator.doMessage(message, context); needMergeComputation = true; } @@ -460,5 +499,6 @@ protected String decodeSQLContent(String sqlContent) { throw new RuntimeException("decode sql content error " + sqlContent, e); } } + } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java index b012b175..e3bb3f83 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java @@ -287,7 +287,6 @@ public int compare(WindowInstance o1, WindowInstance o2) { } }); for (WindowInstance windowInstance : windowInstanceList) { - System.out.println("fire by finish flag"); fireWindowInstance(windowInstance); } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/util/ShuffleUtil.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/util/ShuffleUtil.java new file mode 100644 index 00000000..94d76055 --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/util/ShuffleUtil.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.window.util; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.topology.shuffle.IShuffleKeyGenerator; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.window.model.WindowCache; + +public class ShuffleUtil { + private static final Log LOG = LogFactory.getLog(ShuffleUtil.class); + public static IMessage createShuffleMsg(IMessage msg, String shuffleKey,JSONObject msgHeader){ + if (msg.getHeader().isSystemMessage()) { + return null; + } + + if (StringUtil.isEmpty(shuffleKey)) { + shuffleKey = ""; + LOG.debug("there is no group by value in message! " + msg.getMessageBody().toString()); + //continue; + } + + JSONObject body = msg.getMessageBody(); + String offset = msg.getHeader().getOffset(); + String queueId = msg.getHeader().getQueueId(); + + body.put(WindowCache.ORIGIN_OFFSET, offset); + body.put(WindowCache.ORIGIN_QUEUE_ID,queueId); + body.put(WindowCache.ORIGIN_QUEUE_IS_LONG, msg.getHeader().getMessageOffset().isLongOfMainOffset()); + if(msgHeader==null){ + body.put(WindowCache.ORIGIN_MESSAGE_HEADER, JSONObject.toJSONString(msg.getHeader())); + }else { + body.put(WindowCache.ORIGIN_MESSAGE_HEADER, msgHeader); + } + + body.put(WindowCache.ORIGIN_MESSAGE_TRACE_ID, msg.getHeader().getTraceId()); + body.put(WindowCache.SHUFFLE_KEY, shuffleKey); + return msg; + } + + public static IMessage createShuffleMsg(IMessage msg, String shuffleKey){ + return createShuffleMsg(msg,shuffleKey,null); + } +} diff --git a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/RocksdbTest.java b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/RocksdbTest.java index d173605d..992f0bb3 100644 --- a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/RocksdbTest.java +++ b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/RocksdbTest.java @@ -43,7 +43,7 @@ public class RocksdbTest { public void testCPUCost() throws UnsupportedEncodingException, RocksDBException { RocksDB rocksDB = new RocksDBOperator().getInstance(); long start = System.currentTimeMillis(); - int size = 100000000; + int size = 10000000; for (int i = 0; i < size; i++) { JSONObject msg = new JSONObject(); msg.put("name", "chris" + i);