From ae3396b1ac150745133ab9b72cf5dd6d2546f472 Mon Sep 17 00:00:00 2001 From: yuanxiaodongn Date: Wed, 8 Sep 2021 09:44:59 +0800 Subject: [PATCH 1/5] add logfinger filter strategy fixed bugs for sink batch insert --- .../rocketmq/streams/db/sink/DBSink.java | 102 ++++++++++-------- .../streams/db/sink/DBSinkBuilder.java | 41 ++++--- .../streams/client/DataStreamAction.java | 21 ++++ .../strategy/LogFingerprintStrategy.java | 57 ++++++++++ .../common/cache/compress/BitSetCache.java | 4 + .../cachefilter/CacheFilterGroup.java | 36 +++++-- .../cachefilter/CacheFilterManager.java | 19 +++- .../cachefilter/CacheFilterMeta.java | 43 ++++++++ .../cachefilter/ICacheFilter.java | 6 ++ .../streams/filter/operator/Rule.java | 2 +- .../operator/expression/Expression.java | 11 ++ .../optimization/AbstractExpressionProxy.java | 4 +- .../optimization/EqualsExpressionProxy.java | 4 + .../optimization/LikeExpressionProxy.java | 1 + .../operator/expression/ScriptExpression.java | 42 +++++++- .../script/operator/impl/FunctionScript.java | 5 +- .../ScriptExpressionGroupsProxy.java | 63 +++++------ .../performance/ScriptOptimization.java | 41 +++---- .../SimpleScriptExpressionProxy.java | 10 +- 19 files changed, 359 insertions(+), 153 deletions(-) create mode 100644 rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/LogFingerprintStrategy.java create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterMeta.java diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java index 4a2b50ec..2a7e14b6 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java @@ -16,14 +16,8 @@ */ package org.apache.rocketmq.streams.db.sink; -import com.alibaba.fastjson.JSONObject; -import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.List; -import java.util.Set; + + import org.apache.rocketmq.streams.common.channel.IChannel; import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache; @@ -38,13 +32,29 @@ import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.apache.rocketmq.streams.db.driver.JDBCDriver; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.List; +import java.util.Set; +import com.alibaba.fastjson.JSONObject; +import com.google.common.collect.Lists; + /** * 主要用于写db,输入可以是一个insert/replace 模版,也可以是metadata对象,二者选一即可。都支持批量插入,提高吞吐 sql 模版:insert into table(column1,column2,column3)values('#{var1}',#{var2},'#{var3}') MetaData:主要是描述每个字段的类型,是否必须 二者选一个即可。sql模式,系统会把一批(batchSize)数据拼成一个大sql。metadata模式,基于字段描述,最终也是拼成一个大sql */ public class DBSink extends AbstractSink { - + protected static final Log LOG = LogFactory.getLog(DBSink.class); protected String insertSQLTemplate;//完成插入部分的工作,和metadata二选一。insert into table(column1,column2,column3)values('#{var1}',#{var2},'#{var3}') + protected String duplicateSQLTemplate; //通过on duplicate key update 来对已经存在的信息进行更新 + protected MetaData metaData;//可以指定meta data,和insertSQL二选一 protected String tableName; //指定要插入的数据表 @@ -111,26 +121,29 @@ protected boolean initConfigurable() { ResultSet metaResult = metaData.getColumns(connection.getCatalog(), "%", this.tableName, null); this.metaData = MetaData.createMetaData(metaResult); this.metaData.setTableName(this.tableName); - sqlCache=new MessageCache<>(new IMessageFlushCallBack() { - @Override public boolean flushMessage(List sqls) { - JDBCDriver dataSource = DriverBuilder.createDriver(jdbcDriver, url, userName, password); - try { - dataSource.executSqls(sqls); - } catch (Exception e) { - e.printStackTrace(); - throw new RuntimeException(e); - } finally { - if (dataSource != null) { - dataSource.destroy(); - } + + } + sqlCache = new MessageCache<>(new IMessageFlushCallBack() { + @Override + public boolean flushMessage(List sqls) { + JDBCDriver dataSource = DriverBuilder.createDriver(jdbcDriver, url, userName, password); + try { + LOG.info(getConfigureName()+" DBSink execute sqls "+sqls.size()); + dataSource.executSqls(sqls); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + if (dataSource != null) { + dataSource.destroy(); } - return true; } - }); - ((MessageCache) sqlCache).setAutoFlushTimeGap(100000); - ((MessageCache) sqlCache).setAutoFlushSize(50); - sqlCache.openAutoFlush(); - } + return true; + } + }); + ((MessageCache)sqlCache).setAutoFlushTimeGap(3000); + ((MessageCache)sqlCache).setAutoFlushSize(50); + sqlCache.openAutoFlush(); return super.initConfigurable(); } catch (ClassNotFoundException | SQLException e) { e.printStackTrace(); @@ -160,22 +173,12 @@ protected boolean batchInsert(List messageList) { } return true; } else { - StringBuilder sb = new StringBuilder(); - String insertSQL; - boolean isFirst = true; - int i = 0; + List subInsert = Lists.newArrayList(); for (JSONObject message : messages) { - insertSQL = parseSQL(message, insertValueSQL); - if (isFirst) { - isFirst = false; - } else { - sb.append(","); - } - i++; - - sb.append(insertSQL); + subInsert.add(parseSQL(message, insertValueSQL)); } - insertSQL = this.insertSQLTemplate.replace(insertValueSQL, sb.toString()); + String insertSQL = this.insertSQLTemplate.replace(insertValueSQL, String.join(",", subInsert)); + insertSQL += this.duplicateSQLTemplate; executeSQL(dbDataSource, insertSQL); return true; } @@ -184,17 +187,18 @@ protected boolean batchInsert(List messageList) { } } - @Override public boolean checkpoint(Set splitIds) { - if(sqlCache!=null){ + @Override + public boolean checkpoint(Set splitIds) { + if (sqlCache != null) { sqlCache.flush(splitIds); } return true; } protected void executeSQL(JDBCDriver dbDataSource, String sql) { - if(isOpenSqlCache()){ + if (isOpenSqlCache()) { this.sqlCache.addCache(sql); - }else { + } else { dbDataSource.execute(sql); } @@ -285,4 +289,12 @@ public boolean isOpenSqlCache() { public void setOpenSqlCache(boolean openSqlCache) { this.openSqlCache = openSqlCache; } + + public String getDuplicateSQLTemplate() { + return duplicateSQLTemplate; + } + + public void setDuplicateSQLTemplate(String duplicateSQLTemplate) { + this.duplicateSQLTemplate = duplicateSQLTemplate; + } } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java index c960bae4..87f5fb77 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java @@ -17,15 +17,16 @@ package org.apache.rocketmq.streams.db.sink; import com.google.auto.service.AutoService; -import java.util.List; -import java.util.Properties; +import com.google.common.collect.Lists; 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.metadata.MetaDataField; import org.apache.rocketmq.streams.common.model.ServiceName; -import org.apache.rocketmq.streams.common.utils.DataTypeUtil; + +import java.util.List; +import java.util.Properties; @AutoService(IChannelBuilder.class) @ServiceName(DBSinkBuilder.TYPE) @@ -36,29 +37,23 @@ public class DBSinkBuilder implements IChannelBuilder { public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { DBSink sink = new DBSink(); sink.setUrl(properties.getProperty("url")); - sink.setUserName("userName"); - sink.setPassword("password"); + sink.setUserName(properties.getProperty("userName")); + sink.setPassword(properties.getProperty("password")); List fieldList = metaData.getMetaDataFields(); - StringBuilder insertSQL = new StringBuilder(); - StringBuilder insertValueSQL = new StringBuilder(); - boolean isFirst = true; - for (MetaDataField field : fieldList) { + + List insertFields = Lists.newArrayList(); + List insertValues = Lists.newArrayList(); + List duplicateKeys = Lists.newArrayList(); + fieldList.forEach(field -> { String fieldName = field.getFieldName(); - if (isFirst) { - isFirst = false; - } else { - insertSQL.append(","); - insertValueSQL.append(","); - } - insertSQL.append(fieldName); - if (DataTypeUtil.isNumber(field.getDataType())) { - insertValueSQL.append(fieldName); - } else { - insertValueSQL.append("'#{" + fieldName + "}'"); - } - } - String sql = "insert into " + properties.getProperty("tableName") + "(" + insertSQL.toString() + ")values(" + insertValueSQL.toString() + ")"; + insertFields.add(fieldName); + insertValues.add("'#{" + fieldName + "}'"); + duplicateKeys.add(fieldName + " = VALUES(" + fieldName + ")"); + }); + + String sql = "insert into " + properties.getProperty("tableName") + "(" + String.join(",", insertFields) + ") values (" + String.join(",", insertValues) + ") "; sink.setInsertSQLTemplate(sql); + sink.setDuplicateSQLTemplate(" on duplicate key update " + String.join(",", duplicateKeys)); return sink; } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/DataStreamAction.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/DataStreamAction.java index 91f16fa0..5b9bac1e 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/DataStreamAction.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/DataStreamAction.java @@ -18,13 +18,19 @@ package org.apache.rocketmq.streams.client; import com.google.common.collect.Maps; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; +import org.apache.rocketmq.streams.client.strategy.LogFingerprintStrategy; import org.apache.rocketmq.streams.client.strategy.Strategy; import org.apache.rocketmq.streams.client.transform.DataStream; +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.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.optimization.LogFingerprintFilter; import org.apache.rocketmq.streams.common.topology.ChainPipeline; import org.apache.rocketmq.streams.common.topology.ChainStage; import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder; @@ -45,6 +51,21 @@ public DataStreamAction(PipelineBuilder pipelineBuilder, Set pi public DataStreamAction with(Strategy... strategies) { Properties properties = new Properties(); for (Strategy strategy : strategies) { + + if(strategy instanceof LogFingerprintStrategy){ + ISource source=this.mainPipelineBuilder.getPipeline().getSource(); + if(source instanceof AbstractSource){ + AbstractSource abstractSource=(AbstractSource)source; + String[] logFingerprintFields=((LogFingerprintStrategy)strategy).getLogFingerprintFields(); + if(logFingerprintFields!=null){ + List logFingerprintFieldList=new ArrayList<>(); + for(String loglogFingerprintField:logFingerprintFields){ + logFingerprintFieldList.add(loglogFingerprintField); + } + abstractSource.setLogFingerprintFields(logFingerprintFieldList); + } + } + } properties.putAll(strategy.getStrategyProperties()); } ComponentCreator.createProperties(properties); diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/LogFingerprintStrategy.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/LogFingerprintStrategy.java new file mode 100644 index 00000000..d268ab1d --- /dev/null +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/strategy/LogFingerprintStrategy.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.strategy; + +import java.util.Properties; +import org.apache.rocketmq.streams.common.component.AbstractComponent; + +public class LogFingerprintStrategy implements Strategy { + + protected String[] logFingerprintFields; + + private final Properties properties; + + private LogFingerprintStrategy() { + properties = new Properties(); + } + private LogFingerprintStrategy(String... fieldNames) { + properties = new Properties(); + this.logFingerprintFields=fieldNames; + properties.put("logFingerprintFields",logFingerprintFields); + } + private LogFingerprintStrategy(String url, String username, String password) { + properties = new Properties(); + properties.put(AbstractComponent.JDBC_DRIVER, AbstractComponent.DEFAULT_JDBC_DRIVER); + properties.put(AbstractComponent.JDBC_URL, url); + properties.put(AbstractComponent.JDBC_USERNAME, username); + properties.put(AbstractComponent.JDBC_PASSWORD, password); + properties.put(AbstractComponent.JDBC_TABLE_NAME, AbstractComponent.DEFAULT_JDBC_TABLE_NAME); + } + + public static Strategy configLogFingerprint(String... fieldNames){ + return new LogFingerprintStrategy(fieldNames); + + } + + @Override public Properties getStrategyProperties() { + return this.properties; + } + + public String[] getLogFingerprintFields() { + return logFingerprintFields; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java index a243d8f1..3aad2784 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java @@ -89,4 +89,8 @@ public BitSet get(String key){ } + public long size(){ + return this.cache.getSize(); + } + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterGroup.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterGroup.java index 3bed9119..54f90f84 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterGroup.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterGroup.java @@ -1,7 +1,9 @@ package org.apache.rocketmq.streams.common.optimization.cachefilter; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicLong; import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; import org.apache.rocketmq.streams.common.context.AbstractContext; @@ -12,10 +14,12 @@ * group by var name */ public class CacheFilterGroup { + public static final String FILTER_CACHE_KEY="__filter_cache_key"; //key: varName; value :list IOptimizationExpression protected String name;//mutli FilterOptimization shared cachefilter,need name protected String varName; protected List expressionList=new ArrayList<>(); + protected Map expression2Index=null; protected BitSetCache cache; public CacheFilterGroup(String name,String varName,BitSetCache cache){ this.name=name; @@ -31,7 +35,18 @@ public void addOptimizationExpression(ICacheFilter expression){ public static AtomicLong matchCount=new AtomicLong(0); public void execute(IMessage message, AbstractContext context){ totalCount.incrementAndGet(); - String key= MapKeyUtil.createKey(name,message.getMessageBody().getString(varName)); + if(expression2Index==null){ + synchronized (this){ + if(expression2Index==null){ + expression2Index=new HashMap<>(); + for(int i=0;i0&&totalCount.get()%100==0){ -// -// System.out.println("filter rate is "+(double)matchCount.get()/(double)totalCount.get()); -// } + context.put(createCacheKey(varName),new CacheFilterMeta(varName,name,this.expression2Index,cache )); + if(totalCount.get()>0&&totalCount.get()%10000==0){ + + System.out.println("cache count is "+this.cache.size()+" filter rate is "+(double)matchCount.get()/(double)totalCount.get()); + } + } + + protected String createCacheKey(String varName){ + return MapKeyUtil.createKey(FILTER_CACHE_KEY,varName); } public int getSize(){ diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterManager.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterManager.java index 4697b480..0c65c004 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterManager.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterManager.java @@ -1,13 +1,18 @@ package org.apache.rocketmq.streams.common.optimization.cachefilter; +import java.lang.reflect.Field; import java.util.HashMap; import java.util.Map; import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; + +import static org.apache.rocketmq.streams.common.optimization.cachefilter.CacheFilterGroup.FILTER_CACHE_KEY; public class CacheFilterManager { protected BitSetCache cache; + protected Map filterOptimizationMap=new HashMap<>(); public CacheFilterManager(int elementCount,int capacity){ cache=new BitSetCache(elementCount,capacity); @@ -27,9 +32,19 @@ public void addOptimizationExpression(String name, ICacheFilter expression){ filterOptimization.addOptimizationExpression(expression); } - public void execute(IMessage message, AbstractContext context){ - for(CacheFilterGroup filterOptimization: filterOptimizationMap.values()){ + public void executeExpression(IMessage message, AbstractContext context) { + for(CacheFilterGroup filterOptimization: filterOptimizationMap.values()){ filterOptimization.execute(message,context); + } } + + + protected static String createCacheKey(String varName){ + return MapKeyUtil.createKey(FILTER_CACHE_KEY,varName); + } + + + + } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterMeta.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterMeta.java new file mode 100644 index 00000000..ab4612bc --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/CacheFilterMeta.java @@ -0,0 +1,43 @@ +package org.apache.rocketmq.streams.common.optimization.cachefilter; + +import java.util.BitSet; +import java.util.Map; +import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; +import org.apache.rocketmq.streams.common.context.AbstractContext; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; + +public class CacheFilterMeta { + protected String varName; + protected Map expression2Index; + protected BitSetCache cache; + protected String name; + public CacheFilterMeta(String name,String varName,Map expression2Index, BitSetCache cache){ + this.varName=varName; + this.expression2Index=expression2Index; + this.cache=cache; + this.name=name; + } + + + + + public Boolean match(String varName, String expression, IMessage message, AbstractContext context) { + Integer index=expression2Index.get(expression); + if(index==null){ + return null; + } + BitSetCache.BitSet bitSet = cache.get(createCacheKey(varName,message)); + if(bitSet==null){ + return null; + } + + return bitSet.get(index); + } + + + public static String createCacheKey(String varName,IMessage message){ + String key= MapKeyUtil.createKey(message.getMessageBody().getString(varName)); + return key; + } +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/ICacheFilter.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/ICacheFilter.java index fd7c54c5..ca571777 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/ICacheFilter.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/cachefilter/ICacheFilter.java @@ -30,4 +30,10 @@ public interface ICacheFilter { * @return ori expression */ T getOriExpression(); + + /** + * get expression ,ex:regex string + * @return + */ + String getExpression(); } 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 73938d1d..94907737 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 @@ -125,7 +125,7 @@ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableS initVar(configurableService); initAction(configurableService); initMetaData(configurableService); - //this.optimize(); + // this.optimize(); // groupExpressionManager.compile(); } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/Expression.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/Expression.java index a169f9f9..04f012b1 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/Expression.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/Expression.java @@ -36,9 +36,12 @@ import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.filter.context.RuleContext; import org.apache.rocketmq.streams.filter.function.expression.ExpressionFunction; +import org.apache.rocketmq.streams.filter.function.expression.LikeFunction; +import org.apache.rocketmq.streams.filter.function.expression.RegexFunction; import org.apache.rocketmq.streams.filter.operator.Rule; import org.apache.rocketmq.streams.filter.operator.action.IConfigurableAction; import org.apache.rocketmq.streams.filter.operator.var.Var; +import org.apache.rocketmq.streams.script.optimization.performance.ScriptExpressionGroupsProxy; import org.apache.rocketmq.streams.script.utils.FunctionUtils; public class Expression extends BasedConfigurable @@ -133,6 +136,14 @@ public Boolean doAction(RuleContext context, Rule rule) { } public Boolean getExpressionValue(RuleContext context, Rule rule) { + + if(RegexFunction.isRegex(functionName)|| LikeFunction.isLikeFunciton(functionName)){ + Boolean value=ScriptExpressionGroupsProxy.inFilterCache(getVarName(),getValue().toString(),context.getMessage(),context); + if(value!=null){ + return value; + } + } + Boolean result = context.getExpressionValue(getConfigureName()); if (result != null) { return result; diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/AbstractExpressionProxy.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/AbstractExpressionProxy.java index 5569f790..8a127188 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/AbstractExpressionProxy.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/AbstractExpressionProxy.java @@ -48,6 +48,8 @@ public boolean executeOrigExpression(IMessage message, AbstractContext context) @Override public Expression getOriExpression() { return oriExpression; } - + @Override public String getExpression() { + return this.oriExpression.getValue().toString(); + } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/EqualsExpressionProxy.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/EqualsExpressionProxy.java index f519eb01..eb1003e9 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/EqualsExpressionProxy.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/EqualsExpressionProxy.java @@ -13,4 +13,8 @@ public EqualsExpressionProxy(Expression oriExpression, Rule rule) { @Override public boolean support(Expression oriExpression) { return Equals.isEqualFunction(oriExpression.getFunctionName()) && DataTypeUtil.isString(oriExpression.getDataType()) && oriExpression.getValue() != null; } + + @Override public String getExpression() { + return null; + } } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/LikeExpressionProxy.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/LikeExpressionProxy.java index f66c15d7..856d9599 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/LikeExpressionProxy.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/optimization/LikeExpressionProxy.java @@ -13,4 +13,5 @@ public LikeExpressionProxy(Expression oriExpression, Rule rule) { @Override public boolean support(Expression oriExpression) { return LikeFunction.isLikeFunciton(oriExpression.getFunctionName()) && DataTypeUtil.isString(oriExpression.getDataType()) && oriExpression.getValue() != null; } + } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java index 751bc9d7..84f435ef 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java @@ -31,8 +31,10 @@ import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.script.ScriptComponent; import org.apache.rocketmq.streams.script.context.FunctionContext; +import org.apache.rocketmq.streams.script.function.impl.string.RegexFunction; import org.apache.rocketmq.streams.script.function.model.FunctionConfigure; import org.apache.rocketmq.streams.script.optimization.compile.CompileScriptExpression; +import org.apache.rocketmq.streams.script.optimization.performance.ScriptExpressionGroupsProxy; import org.apache.rocketmq.streams.script.service.IScriptExpression; import org.apache.rocketmq.streams.script.service.IScriptParamter; import org.apache.rocketmq.streams.script.utils.FunctionUtils; @@ -68,13 +70,37 @@ public Object executeExpression(IMessage message, FunctionContext context) { FunctionUtils.getValue(message, context, value.toString())); return value; } + Object value = null; + + + if(RegexFunction.isRegexFunction(functionName)){ + ScriptParameter varParameter=(ScriptParameter) getParameters().get(0); + ScriptParameter regexParameter=(ScriptParameter) getParameters().get(1); + String varName=getParameterValue(varParameter); + String expression=getParameterValue(regexParameter); + if(varName!=null||expression!=null){ + value= ScriptExpressionGroupsProxy.inFilterCache(varName,expression,message,context); + } + + + } + if(value!=null){ + if (StringUtil.isNotEmpty(newFieldName) && value != null) { + setValue2Var(message, context, newFieldName, value); + } + return value; + } if (compileScriptExpression != null) { value = compileScriptExpression.execute(message, context); } else { value = execute(message, context); } + if (StringUtil.isNotEmpty(newFieldName) && value != null) { + setValue2Var(message, context, newFieldName, value); + } + //monitor.setResult(value); //monitor.endMonitor(); //if (monitor.isSlow()) { @@ -96,6 +122,17 @@ public Object executeExpression(IMessage message, FunctionContext context) { } + protected String getParameterValue(IScriptParamter scriptParamter) { + if (!ScriptParameter.class.isInstance(scriptParamter)) { + return null; + } + ScriptParameter parameter = (ScriptParameter)scriptParamter; + if (parameter.getRigthVarName() != null) { + return null; + } + return FunctionUtils.getConstant(parameter.getLeftVarName()); + } + private ScriptComponent scriptComponent = ScriptComponent.getInstance(); public Object execute(IMessage message, FunctionContext context) { @@ -124,10 +161,7 @@ public Object execute(IMessage message, FunctionContext context) { } Object value = functionConfigure.execute(ps); compileScriptExpression = new CompileScriptExpression(this, functionConfigure); - if (StringUtil.isNotEmpty(newFieldName) && value != null) { - setValue2Var(message, context, newFieldName, value); - //message.getMessageBody().put(newFieldName, value); - } + return value; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/FunctionScript.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/FunctionScript.java index e95529eb..4a21b4cb 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/FunctionScript.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/FunctionScript.java @@ -38,6 +38,7 @@ import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.script.context.FunctionContext; import org.apache.rocketmq.streams.script.operator.expression.ScriptExpression; +import org.apache.rocketmq.streams.script.optimization.performance.ScriptExpressionGroupsProxy; import org.apache.rocketmq.streams.script.optimization.performance.ScriptOptimization; import org.apache.rocketmq.streams.script.parser.imp.FunctionParser; import org.apache.rocketmq.streams.script.service.IScriptExpression; @@ -54,7 +55,7 @@ public class FunctionScript extends AbstractScript, FunctionConte * 脚本解析的表达式列表 */ private transient List scriptExpressions = new ArrayList(); - + protected transient ScriptExpressionGroupsProxy scriptExpressionGroupsProxy; /** * 表达式,转化成streamoperator接口列表,可以在上层中使用 */ @@ -88,7 +89,7 @@ protected boolean initConfigurable() { //表达式优化,在运行中收集信息,减少解析查找的时间 ScriptOptimization scriptOptimization = new ScriptOptimization(MapKeyUtil.createKey(getNameSpace(),getConfigureName()),this.scriptExpressions); if (scriptOptimization.supportOptimize()) { - expressions = scriptOptimization.optimize(); + scriptExpressionGroupsProxy= scriptOptimization.optimize(); } //转化成istreamoperator 接口 diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptExpressionGroupsProxy.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptExpressionGroupsProxy.java index d083078d..722921f6 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptExpressionGroupsProxy.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptExpressionGroupsProxy.java @@ -5,19 +5,40 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.rocketmq.streams.common.cache.compress.BitSetCache; +import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.optimization.cachefilter.CacheFilterGroup; import org.apache.rocketmq.streams.common.optimization.cachefilter.CacheFilterManager; -import org.apache.rocketmq.streams.script.context.FunctionContext; +import org.apache.rocketmq.streams.common.optimization.cachefilter.CacheFilterMeta; import org.apache.rocketmq.streams.script.service.IScriptExpression; -import org.apache.rocketmq.streams.script.service.IScriptParamter; -public class ScriptExpressionGroupsProxy extends CacheFilterManager implements IScriptExpression { +public class ScriptExpressionGroupsProxy extends CacheFilterManager { protected List scriptExpressions=new ArrayList<>(); + public ScriptExpressionGroupsProxy(int elementCount, int capacity) { super(elementCount, capacity); } + + public static Boolean inFilterCache(String varName, String expression, IMessage message, AbstractContext context) { + CacheFilterMeta cacheFilterMeta=getCacheFilterMeta(varName,context); + if(cacheFilterMeta==null){ + return null; + } + return cacheFilterMeta.match(varName,expression,message,context); + } + + public static void setFilterCache(String varName, String expression, IMessage message, AbstractContext context){ + + } + + private static CacheFilterMeta getCacheFilterMeta(String varName, AbstractContext context) { + String key=createCacheKey(varName); + return (CacheFilterMeta) context.get(key); + } + + public void removeLessCount() { Map newFilterOptimizationMap=new HashMap<>(); for(String varName:this.filterOptimizationMap.keySet()){ @@ -31,41 +52,5 @@ public void removeLessCount() { public void addScriptExpression(IScriptExpression scriptExpression){ this.scriptExpressions.add(scriptExpression); } - @Override public Object executeExpression(IMessage message, FunctionContext context) { - this.execute(message,context); - for(IScriptExpression scriptExpression:scriptExpressions){ - scriptExpression.executeExpression(message,context); - } - return null; - } - - @Override public List getScriptParamters() { - return null; - } - - @Override public String getFunctionName() { - return null; - } - - @Override public String getExpressionDescription() { - return null; - } - - @Override public Object getScriptParamter(IMessage message, FunctionContext context) { - return null; - } - - @Override public String getScriptParameterStr() { - return null; - } - - @Override public List getDependentFields() { - return null; - } - - @Override public Set getNewFieldNames() { - return null; - } - } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptOptimization.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptOptimization.java index ae147ee4..c23b359f 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptOptimization.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptOptimization.java @@ -96,15 +96,15 @@ public boolean supportOptimize() { /** * 把表达式拆成3段,创建变量的,正则类,其他。正则类用HyperscanRegex做优化 */ - public List optimize() { + public ScriptExpressionGroupsProxy optimize() { if (!startOptimization.compareAndSet(false, true)) { - return this.scriptExpressions; + return this.scriptExpressionGroupsProxy; } Set newVarNames=new HashSet<>(); - List allScriptExpressions = new ArrayList<>();//最终输出的表达式列表 - List proxyExpressions = new ArrayList<>();//最后执行的脚本,在执行完正则后执行的部分 - List lastExpressions = new ArrayList<>();//最后执行的脚本,在执行完正则后执行的部分 - List mapExpressions = new ArrayList<>();//如果是trim,cast,concat等函数,优先执行 +// List allScriptExpressions = new ArrayList<>();//最终输出的表达式列表 +// List proxyExpressions = new ArrayList<>();//最后执行的脚本,在执行完正则后执行的部分 +// List lastExpressions = new ArrayList<>();//最后执行的脚本,在执行完正则后执行的部分 +// List mapExpressions = new ArrayList<>();//如果是trim,cast,concat等函数,优先执行 for (IScriptExpression scriptExpression : scriptExpressions) { Set newFieldNames = scriptExpression.getNewFieldNames(); @@ -113,29 +113,22 @@ public List optimize() { newVarNames.add(newFieldName); } + createProxy(scriptExpression,newVarNames); - - IScriptExpression scriptExpressionProxy = createProxy(scriptExpression,newVarNames); - String functionName = scriptExpressionProxy.getFunctionName(); - if(scriptExpressionProxy instanceof AbstractScriptProxy){ - proxyExpressions.add(scriptExpressionProxy); - }else if("trim".equals(functionName) || "lower".equals(functionName) || "concat".equals(functionName)){ - mapExpressions.add(scriptExpressionProxy); - }else { - lastExpressions.add(scriptExpressionProxy); - } +// IScriptExpression scriptExpressionProxy = +// String functionName = scriptExpressionProxy.getFunctionName(); +// if(scriptExpressionProxy instanceof AbstractScriptProxy){ +// // proxyExpressions.add(scriptExpressionProxy); +// }else if("trim".equals(functionName) || "lower".equals(functionName) || "concat".equals(functionName)){ +// // mapExpressions.add(scriptExpressionProxy); +// }else { +// //lastExpressions.add(scriptExpressionProxy); +// } } - allScriptExpressions.addAll(mapExpressions);//把优先执行的表达式添加上 - if(this.scriptExpressionGroupsProxy.scriptExpressions.size()>0){ - allScriptExpressions.add(this.scriptExpressionGroupsProxy); - } - allScriptExpressions.addAll(lastExpressions);//把剩余的表达式增加到list中 - - this.scriptExpressions=allScriptExpressions; this.scriptExpressionGroupsProxy.removeLessCount(); - return this.scriptExpressions; + return scriptExpressionGroupsProxy; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/SimpleScriptExpressionProxy.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/SimpleScriptExpressionProxy.java index 247d9661..73cacb24 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/SimpleScriptExpressionProxy.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/SimpleScriptExpressionProxy.java @@ -7,7 +7,10 @@ import org.apache.rocketmq.streams.common.optimization.cachefilter.AbstractCacheFilter; import org.apache.rocketmq.streams.common.optimization.cachefilter.ICacheFilter; import org.apache.rocketmq.streams.script.context.FunctionContext; +import org.apache.rocketmq.streams.script.operator.expression.ScriptParameter; import org.apache.rocketmq.streams.script.service.IScriptExpression; +import org.apache.rocketmq.streams.script.service.IScriptParamter; +import org.apache.rocketmq.streams.script.utils.FunctionUtils; public abstract class SimpleScriptExpressionProxy extends AbstractScriptProxy { @@ -22,7 +25,7 @@ public List getCacheFilters() { synchronized (this){ if(this.optimizationExpressions==null){ List optimizationExpressions=new ArrayList<>(); - optimizationExpressions.add(new AbstractCacheFilter(getVarName(),this.origExpression) { + optimizationExpressions.add(new AbstractCacheFilter(getVarName(),this.origExpression) { @Override public boolean executeOrigExpression(IMessage message, AbstractContext context) { FunctionContext functionContext = new FunctionContext(message); if (context != null) { @@ -35,6 +38,10 @@ public List getCacheFilters() { } return isMatch; } + + @Override public String getExpression() { + return getParameterValue((IScriptParamter)origExpression.getScriptParamters().get(1)); + } }); this.optimizationExpressions=optimizationExpressions; } @@ -45,6 +52,7 @@ public List getCacheFilters() { } + @Override public Object executeExpression(IMessage message, FunctionContext context) { Boolean value= this.optimizationExpressions.get(0).execute(message,context); if(this.origExpression.getNewFieldNames()!=null&&this.origExpression.getNewFieldNames().size()>0){ From 7752a44aa45a2a174d71cfe62db053630043354c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=9B=B6=E5=8F=B7=E7=A8=8B=E5=BA=8F?= Date: Wed, 15 Sep 2021 11:36:58 +0800 Subject: [PATCH 2/5] =?UTF-8?q?add=20update=20logic=20for=20the=20DBSinker?= =?UTF-8?q?=20=E3=80=81=20upgrade=20the=20concat=5Fws=20function=20(#57)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * add update logic for the DBSinker 、 upgrade the concat_ws function * Add the field level cache to reduce duplicate data entry #60 Co-authored-by: junjie.cheng --- .../rocketmq/streams/db/sink/DBSink.java | 96 ++++++------- .../streams/db/sink/DBSinkBuilder.java | 41 +++--- .../streams/client/DataStreamAction.java | 6 +- .../transform/window/HoppingWindow.java | 5 +- .../common/cache/compress/AdditionStore.java | 7 +- .../common/cache/compress/BitSetCache.java | 88 ++++++------ .../common/cache/compress/ByteArray.java | 13 +- .../cache/compress/ByteArrayValueKV.java | 9 +- .../common/cache/compress/CacheKV.java | 18 +-- .../common/cache/compress/ICacheKV.java | 1 - .../common/cache/compress/KVElement.java | 2 +- .../compress/impl/FixedLenRowCacheKV.java | 4 +- .../cache/compress/impl/IntValueKV.java | 8 +- .../cache/compress/impl/LongValueKV.java | 67 ++++++++++ .../cache/compress/impl/MutilValueKV.java | 46 +++---- .../AbstractSupportShuffleChannelBuilder.java | 2 +- .../channel/impl/OutputPrintChannel.java | 7 +- .../common/channel/sink/AbstractSink.java | 35 ++--- .../common/context/AbstractContext.java | 19 +-- .../common/topology/ChainPipeline.java | 95 ++++++++++--- .../topology/stages/OutputChainStage.java | 52 ++++---- .../streams/common/utils/SQLUtil.java | 54 ++++---- .../service/AbstractConfigurableService.java | 6 +- .../streams/filter/context/RuleContext.java | 2 +- .../service/storages/DBLeaseStorage.java | 1 + .../script/context/FunctionContext.java | 2 +- .../function/impl/field/FieldFunction.java | 48 ++++--- .../impl/flatmap/SplitArrayFunction.java | 4 +- .../performance/AbstractScriptProxy.java | 11 +- .../CaseScriptExpressionProxy.java | 31 ++--- .../optimization/performance/EqualsProxy.java | 5 +- .../optimization/performance/RegexProxy.java | 8 +- .../ScriptExpressionGroupsProxy.java | 26 ++-- .../performance/ScriptOptimization.java | 62 +++------ .../performance/ScriptProxyFactory.java | 26 ++-- .../SimpleScriptExpressionProxy.java | 26 ++-- .../window/operator/AbstractWindow.java | 126 +++++++++--------- .../streams/window/sqlcache/SQLCache.java | 9 +- 38 files changed, 594 insertions(+), 474 deletions(-) create mode 100644 rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongValueKV.java diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java index 4a2b50ec..1aab2bab 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java @@ -17,13 +17,7 @@ package org.apache.rocketmq.streams.db.sink; import com.alibaba.fastjson.JSONObject; -import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.List; -import java.util.Set; +import com.google.common.collect.Lists; import org.apache.rocketmq.streams.common.channel.IChannel; import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache; @@ -38,6 +32,14 @@ import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.apache.rocketmq.streams.db.driver.JDBCDriver; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.List; +import java.util.Set; + /** * 主要用于写db,输入可以是一个insert/replace 模版,也可以是metadata对象,二者选一即可。都支持批量插入,提高吞吐 sql 模版:insert into table(column1,column2,column3)values('#{var1}',#{var2},'#{var3}') MetaData:主要是描述每个字段的类型,是否必须 二者选一个即可。sql模式,系统会把一批(batchSize)数据拼成一个大sql。metadata模式,基于字段描述,最终也是拼成一个大sql */ @@ -45,6 +47,8 @@ public class DBSink extends AbstractSink { protected String insertSQLTemplate;//完成插入部分的工作,和metadata二选一。insert into table(column1,column2,column3)values('#{var1}',#{var2},'#{var3}') + protected String duplicateSQLTemplate; //通过on duplicate key update 来对已经存在的信息进行更新 + protected MetaData metaData;//可以指定meta data,和insertSQL二选一 protected String tableName; //指定要插入的数据表 @@ -58,7 +62,7 @@ public class DBSink extends AbstractSink { @ENVDependence protected String password; - protected boolean openSqlCache=false; + protected boolean openSqlCache = true; protected transient IMessageCache sqlCache;//cache sql, batch submit sql @@ -111,26 +115,27 @@ protected boolean initConfigurable() { ResultSet metaResult = metaData.getColumns(connection.getCatalog(), "%", this.tableName, null); this.metaData = MetaData.createMetaData(metaResult); this.metaData.setTableName(this.tableName); - sqlCache=new MessageCache<>(new IMessageFlushCallBack() { - @Override public boolean flushMessage(List sqls) { - JDBCDriver dataSource = DriverBuilder.createDriver(jdbcDriver, url, userName, password); - try { - dataSource.executSqls(sqls); - } catch (Exception e) { - e.printStackTrace(); - throw new RuntimeException(e); - } finally { - if (dataSource != null) { - dataSource.destroy(); - } + } + sqlCache = new MessageCache<>(new IMessageFlushCallBack() { + @Override + public boolean flushMessage(List sqls) { + JDBCDriver dataSource = DriverBuilder.createDriver(jdbcDriver, url, userName, password); + try { + dataSource.executSqls(sqls); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + if (dataSource != null) { + dataSource.destroy(); } - return true; } - }); - ((MessageCache) sqlCache).setAutoFlushTimeGap(100000); - ((MessageCache) sqlCache).setAutoFlushSize(50); - sqlCache.openAutoFlush(); - } + return true; + } + }); + ((MessageCache) sqlCache).setAutoFlushTimeGap(100000); + ((MessageCache) sqlCache).setAutoFlushSize(50); + sqlCache.openAutoFlush(); return super.initConfigurable(); } catch (ClassNotFoundException | SQLException e) { e.printStackTrace(); @@ -148,7 +153,8 @@ protected boolean batchInsert(List messageList) { List messages = convertJsonObjectFromMessage(messageList); if (StringUtil.isEmpty(insertSQLTemplate) && metaData != null) { String sql = SQLUtil.createInsertSql(metaData, messages.get(0)); - sql = sql + SQLUtil.createInsertValuesSQL(metaData, messages.subList(1, messages.size())); + sql += SQLUtil.createInsertValuesSQL(metaData, messages.subList(1, messages.size())); + sql += this.duplicateSQLTemplate; executeSQL(dbDataSource, sql); return true; } @@ -156,26 +162,17 @@ protected boolean batchInsert(List messageList) { if (StringUtil.isEmpty(insertValueSQL) || insertSQLTemplate.replace(insertValueSQL, "").contains("#{")) { for (JSONObject message : messages) { String sql = parseSQL(message, insertSQLTemplate); + sql += this.duplicateSQLTemplate; executeSQL(dbDataSource, sql); } return true; } else { - StringBuilder sb = new StringBuilder(); - String insertSQL; - boolean isFirst = true; - int i = 0; + List subInsert = Lists.newArrayList(); for (JSONObject message : messages) { - insertSQL = parseSQL(message, insertValueSQL); - if (isFirst) { - isFirst = false; - } else { - sb.append(","); - } - i++; - - sb.append(insertSQL); + subInsert.add(parseSQL(message, insertValueSQL)); } - insertSQL = this.insertSQLTemplate.replace(insertValueSQL, sb.toString()); + String insertSQL = this.insertSQLTemplate.replace(insertValueSQL, String.join(",", subInsert)); + insertSQL += this.duplicateSQLTemplate; executeSQL(dbDataSource, insertSQL); return true; } @@ -184,17 +181,18 @@ protected boolean batchInsert(List messageList) { } } - @Override public boolean checkpoint(Set splitIds) { - if(sqlCache!=null){ + @Override + public boolean checkpoint(Set splitIds) { + if (sqlCache != null) { sqlCache.flush(splitIds); } return true; } protected void executeSQL(JDBCDriver dbDataSource, String sql) { - if(isOpenSqlCache()){ + if (isOpenSqlCache()) { this.sqlCache.addCache(sql); - }else { + } else { dbDataSource.execute(sql); } @@ -285,4 +283,12 @@ public boolean isOpenSqlCache() { public void setOpenSqlCache(boolean openSqlCache) { this.openSqlCache = openSqlCache; } + + public String getDuplicateSQLTemplate() { + return duplicateSQLTemplate; + } + + public void setDuplicateSQLTemplate(String duplicateSQLTemplate) { + this.duplicateSQLTemplate = duplicateSQLTemplate; + } } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java index c960bae4..87f5fb77 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java @@ -17,15 +17,16 @@ package org.apache.rocketmq.streams.db.sink; import com.google.auto.service.AutoService; -import java.util.List; -import java.util.Properties; +import com.google.common.collect.Lists; 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.metadata.MetaDataField; import org.apache.rocketmq.streams.common.model.ServiceName; -import org.apache.rocketmq.streams.common.utils.DataTypeUtil; + +import java.util.List; +import java.util.Properties; @AutoService(IChannelBuilder.class) @ServiceName(DBSinkBuilder.TYPE) @@ -36,29 +37,23 @@ public class DBSinkBuilder implements IChannelBuilder { public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) { DBSink sink = new DBSink(); sink.setUrl(properties.getProperty("url")); - sink.setUserName("userName"); - sink.setPassword("password"); + sink.setUserName(properties.getProperty("userName")); + sink.setPassword(properties.getProperty("password")); List fieldList = metaData.getMetaDataFields(); - StringBuilder insertSQL = new StringBuilder(); - StringBuilder insertValueSQL = new StringBuilder(); - boolean isFirst = true; - for (MetaDataField field : fieldList) { + + List insertFields = Lists.newArrayList(); + List insertValues = Lists.newArrayList(); + List duplicateKeys = Lists.newArrayList(); + fieldList.forEach(field -> { String fieldName = field.getFieldName(); - if (isFirst) { - isFirst = false; - } else { - insertSQL.append(","); - insertValueSQL.append(","); - } - insertSQL.append(fieldName); - if (DataTypeUtil.isNumber(field.getDataType())) { - insertValueSQL.append(fieldName); - } else { - insertValueSQL.append("'#{" + fieldName + "}'"); - } - } - String sql = "insert into " + properties.getProperty("tableName") + "(" + insertSQL.toString() + ")values(" + insertValueSQL.toString() + ")"; + insertFields.add(fieldName); + insertValues.add("'#{" + fieldName + "}'"); + duplicateKeys.add(fieldName + " = VALUES(" + fieldName + ")"); + }); + + String sql = "insert into " + properties.getProperty("tableName") + "(" + String.join(",", insertFields) + ") values (" + String.join(",", insertValues) + ") "; sink.setInsertSQLTemplate(sql); + sink.setDuplicateSQLTemplate(" on duplicate key update " + String.join(",", duplicateKeys)); return sink; } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/DataStreamAction.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/DataStreamAction.java index 91f16fa0..0a052fab 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/DataStreamAction.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/DataStreamAction.java @@ -18,9 +18,11 @@ package org.apache.rocketmq.streams.client; import com.google.common.collect.Maps; + import java.util.Map; import java.util.Properties; import java.util.Set; + import org.apache.rocketmq.streams.client.strategy.Strategy; import org.apache.rocketmq.streams.client.transform.DataStream; import org.apache.rocketmq.streams.common.component.ComponentCreator; @@ -79,11 +81,11 @@ protected void start(boolean isAsync) { } ConfigurableComponent configurableComponent = ComponentCreator.getComponent(mainPipelineBuilder.getPipelineNameSpace(), ConfigurableComponent.class, kvs); - ChainPipeline pipeline = this.mainPipelineBuilder.build(configurableComponent.getService()); + ChainPipeline pipeline = this.mainPipelineBuilder.build(configurableComponent.getService()); pipeline.startChannel(); if (this.otherPipelineBuilders != null) { for (PipelineBuilder builder : otherPipelineBuilders) { - ChainPipeline otherPipeline = builder.build(configurableComponent.getService()); + ChainPipeline otherPipeline = builder.build(configurableComponent.getService()); otherPipeline.startChannel(); } } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/HoppingWindow.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/HoppingWindow.java index 0c945f0c..9078b16b 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/HoppingWindow.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/HoppingWindow.java @@ -20,10 +20,11 @@ public class HoppingWindow { /** * 滑动窗口信息 + * * @return */ - public static WindowInfo of(Time windowSize,Time windowSlide){ - WindowInfo windowInfo=new WindowInfo(); + public static WindowInfo of(Time windowSize, Time windowSlide) { + WindowInfo windowInfo = new WindowInfo(); windowInfo.setType(WindowInfo.HOPPING_WINDOW); windowInfo.setWindowSize(windowSize); windowInfo.setWindowSlide(windowSlide); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java index 0b441bf4..e8485b7d 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/AdditionStore.java @@ -19,7 +19,6 @@ import java.util.ArrayList; import java.util.List; - public class AdditionStore { /** @@ -104,8 +103,8 @@ public CacheKV.MapAddress add2Store(byte[] value) { CacheKV.MapAddress address = new CacheKV.MapAddress(conflictIndex, conflictOffset); if (isVarLen) { int size = value.length; - bytes[conflictOffset] = (byte)(size & 0xff); - bytes[conflictOffset + 1] = (byte)(size >> 8 & 0xff); + bytes[conflictOffset] = (byte) (size & 0xff); + bytes[conflictOffset + 1] = (byte) (size >> 8 & 0xff); conflictOffset = conflictOffset + 2; } for (int i = 0; i < value.length; i++) { @@ -131,7 +130,7 @@ public ByteArray getValue(CacheKV.MapAddress mapAddress) { if (bytes == null) { return null; } - if (isVarLen == false) { + if (!isVarLen) { return new ByteArray(bytes, mapAddress.offset, elementSize); } else { int len = new ByteArray(bytes, mapAddress.offset, 2).castInt(0, 2); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java index 7a4c5fbc..bc4a6f16 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/BitSetCache.java @@ -26,81 +26,81 @@ public class BitSetCache { protected int capacity; protected int bitSetSize; - public class BitSet{ + public class BitSet { private byte[] bytes; - public BitSet(){ - bytes=new byte[byteSetSize]; + public BitSet() { + bytes = new byte[byteSetSize]; } - public BitSet(byte[] bytes){ - this.bytes=bytes; + + public BitSet(byte[] bytes) { + this.bytes = bytes; } - public void set(int index){ - if(index>bitSetSize){ - throw new RuntimeException("the index exceed max index, max index is "+byteSetSize+", real is "+index); + + public void set(int index) { + if (index > bitSetSize) { + throw new RuntimeException("the index exceed max index, max index is " + byteSetSize + ", real is " + index); } - int byteIndex=index/8; - int bitIndex=index%8; - byte byteElement=bytes[byteIndex]; - byteElement = (byte) (byteElement|(1 << bitIndex)); - bytes[byteIndex]=byteElement; + int byteIndex = index / 8; + int bitIndex = index % 8; + byte byteElement = bytes[byteIndex]; + byteElement = (byte) (byteElement | (1 << bitIndex)); + bytes[byteIndex] = byteElement; } - public boolean get(int index){ - if(index>bitSetSize){ - throw new RuntimeException("the index exceed max index, max index is "+byteSetSize+", real is "+index); + + public boolean get(int index) { + if (index > bitSetSize) { + throw new RuntimeException("the index exceed max index, max index is " + byteSetSize + ", real is " + index); } - int byteIndex=index/8; - int bitIndex=index%8; - byte byteElement=bytes[byteIndex]; - boolean isTrue = ((byteElement & (1 << bitIndex)) != 0); - return isTrue; + int byteIndex = index / 8; + int bitIndex = index % 8; + byte byteElement = bytes[byteIndex]; + return ((byteElement & (1 << bitIndex)) != 0); } - public byte[] getBytes(){ + public byte[] getBytes() { return bytes; } } - public BitSet createBitSet(){ + public BitSet createBitSet() { return new BitSet(); } - - public BitSetCache(int bitSetSize, int capacity){ - cache=new ByteArrayValueKV(capacity,true); - this.byteSetSize=bitSetSize/8+bitSetSize%8; - this.capacity=capacity; - this.bitSetSize=bitSetSize; + public BitSetCache(int bitSetSize, int capacity) { + cache = new ByteArrayValueKV(capacity, true); + this.byteSetSize = bitSetSize / 8 + bitSetSize % 8; + this.capacity = capacity; + this.bitSetSize = bitSetSize; } - - public void put(String key,BitSet bitSet){ - if(cache.size>cache.capacity){ - synchronized (this){ - if(cache.size>cache.capacity){ - cache=new ByteArrayValueKV(capacity,true); + public void put(String key, BitSet bitSet) { + if (cache.size > cache.capacity) { + synchronized (this) { + if (cache.size > cache.capacity) { + cache = new ByteArrayValueKV(capacity, true); } } } - cache.put(key,bitSet.getBytes()); + cache.put(key, bitSet.getBytes()); } public static void main(String[] args) { - BitSetCache bitSetCache=new BitSetCache(150,30000); - BitSet bitSet=bitSetCache.createBitSet(); + BitSetCache bitSetCache = new BitSetCache(150, 30000); + BitSet bitSet = bitSetCache.createBitSet(); bitSet.set(13); - bitSetCache.put("fdsdf",bitSet); - BitSet bitSet1=bitSetCache.get("fdsdf"); + bitSetCache.put("fdsdf", bitSet); + BitSet bitSet1 = bitSetCache.get("fdsdf"); System.out.println(bitSet1.get(13)); } - public BitSet get(String key){ - byte[] bytes=cache.get(key); - if(bytes==null){ + public BitSet get(String key) { + byte[] bytes = cache.get(key); + if (bytes == null) { return null; } - return new BitSet(bytes); + return new BitSet(bytes); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java index 969d2998..cc48286e 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArray.java @@ -59,6 +59,15 @@ public int castInt(int offset, int size) { return res; } + public long castLong(int offset, int size) { + int index = startIndex + offset; + long res = 0L; + for (int i = 0; i < size; i++) { + res += (long) (bytes[i + index] & 0xff) << (i * 8); + } + return res; + } + public byte getByte(int offset) { int index = startIndex + offset; return bytes[index]; @@ -94,9 +103,9 @@ protected void flush(byte[] bytes) { protected void flush(int value) { for (int i = 0; i < 4; i++) { if (i == 0) { - this.bytes[i + this.startIndex] = (byte)(value & 0xff); + this.bytes[i + this.startIndex] = (byte) (value & 0xff); } else { - this.bytes[i + this.startIndex] = (byte)(value >> (i * 8) & 0xff); + this.bytes[i + this.startIndex] = (byte) (value >> (i * 8) & 0xff); } } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java index 0e3d7895..6d3bd38b 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ByteArrayValueKV.java @@ -89,18 +89,13 @@ public void put(String key, byte[] value) { @Override public int calMemory() { - int value = super.calMemory() + (this.conflicts.getConflictIndex() + 1) * this.conflicts - .getBlockSize(); - return value; + return super.calMemory() + (this.conflicts.getConflictIndex() + 1) * this.conflicts.getBlockSize(); } @Override public boolean contains(String key) { byte[] bytes = get(key); - if (bytes == null) { - return false; - } - return true; + return bytes != null; } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java index 08bb056c..4af9ba1a 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/CacheKV.java @@ -75,7 +75,9 @@ public CacheKV(int capacity) { public abstract void put(String key, T value); public T remove(String key) { - if (StringUtil.isEmpty(key)) { return null; } + if (StringUtil.isEmpty(key)) { + return null; + } MapElementContext context = queryMapElementByHashCode(key); /** * TODO: @@ -113,11 +115,11 @@ public boolean putInner(String key, int value, boolean supportUpdate) { KVElement mapElement = context.mapElement; //如果没有发生冲突,说明当前节点无被占用,直接写入 - if (context.isOccurConflict == false) { + if (!context.isOccurConflict) { size++; mapElement.keyHashCode.flush(mapElement.getKeyHashCode()); - if (mapElement.isNoValue() == false) { + if (!mapElement.isNoValue()) { mapElement.value.flush(value); } @@ -125,7 +127,7 @@ public boolean putInner(String key, int value, boolean supportUpdate) { } else { //如果key已经存在,覆盖value if (context.isMatchKey) { - if (mapElement.isNoValue() == false) { + if (!mapElement.isNoValue()) { if (!supportUpdate) { return false; } @@ -280,7 +282,7 @@ public MapElementContext(KVElement mapElement, MapAddress mapAddress, boolean is this.mapAddress = mapAddress; this.mapElement = mapElement; this.isMatchKey = isMatchKey; - if (mapElement.isEmpty() == false) { + if (!mapElement.isEmpty()) { isOccurConflict = true; } } @@ -327,7 +329,7 @@ public MapAddress() { } public boolean isEmpty() { - return isConflict == false && conflictIndex == 0 && offset == 0; + return !isConflict && conflictIndex == 0 && offset == 0; } /** @@ -356,14 +358,14 @@ public MapAddress(ByteArray byteArray) { public byte[] createBytes() { byte[] bytes = NumberUtils.toByte(offset); int value = 0; - byte fisrtByte = (byte)(conflictIndex & 0xff); + byte fisrtByte = (byte) (conflictIndex & 0xff); if (isConflict) { value = (fisrtByte | (1 << 7));//把第一位变成1 } else { return bytes; } - bytes[bytes.length - 1] = (byte)(value & 0xff); + bytes[bytes.length - 1] = (byte) (value & 0xff); return bytes; } } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java index 9a766e03..885affc6 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/ICacheKV.java @@ -18,7 +18,6 @@ /** * kv提供的对外接口,通过二进制实现存储,减少java对象带来的头部开销。 需要指定初始容量,会在创建对象时分配内存。 - * */ public interface ICacheKV { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java index 60f2f0bd..d494882f 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/KVElement.java @@ -51,7 +51,7 @@ public KVElement(ByteArray byteArray) { } public static byte[] createByteArray(CacheKV.MapAddress nextAddress, byte[] keyHashCode, int value, - int elementSize) { + int elementSize) { KVElement element = new KVElement(nextAddress, keyHashCode, value); element.setElementSize(elementSize); return element.getBytes(); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/FixedLenRowCacheKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/FixedLenRowCacheKV.java index 56a300fa..d595da35 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/FixedLenRowCacheKV.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/FixedLenRowCacheKV.java @@ -133,8 +133,8 @@ public byte[] createBytes(Object... values) { Object o = values[i]; byte[] byteValue = null; if (dataType instanceof SetDataType) { - byteValue = new byte[] {(byte)i}; - add2Set(i, (Set)o); + byteValue = new byte[] {(byte) i}; + add2Set(i, (Set) o); } else { byteValue = dataType.toBytes(o, false); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java index a7577d35..23b354ed 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/IntValueKV.java @@ -35,7 +35,6 @@ /** * 支持key是string,value是int的场景,支持size不大于10000000.只支持int,long,boolean,string类型 - * */ public class IntValueKV extends CacheKV { @@ -79,9 +78,7 @@ public boolean contains(String key) { @Override public int calMemory() { - int value = super.calMemory() + (this.conflicts.getConflictIndex() + 1) * this.conflicts - .getBlockSize(); - return value; + return super.calMemory() + (this.conflicts.getConflictIndex() + 1) * this.conflicts.getBlockSize(); } /** @@ -89,9 +86,6 @@ public int calMemory() { * * @return */ - //public Integer remove(String key) { - // return null; - //} public IntValueKV(int capacity) { super(capacity); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongValueKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongValueKV.java new file mode 100644 index 00000000..78bd0329 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/LongValueKV.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.cache.compress.impl; + +import java.nio.ByteBuffer; +import org.apache.rocketmq.streams.common.cache.compress.ByteArrayValueKV; +import org.apache.rocketmq.streams.common.cache.compress.CacheKV; + +public class LongValueKV extends CacheKV { + + private final ByteArrayValueKV byteArrayValueKV; + + public LongValueKV(int capacity) { + super(capacity, 8); + byteArrayValueKV = new ByteArrayValueKV(capacity, true); + } + + @Override + public void put(String key, Long value) { + ByteBuffer buffer = ByteBuffer.allocate(8); + buffer.putLong(0, value); + byte[] bytes = buffer.array(); + byteArrayValueKV.put(key, bytes); + } + + @Override + public boolean contains(String key) { + return byteArrayValueKV.contains(key); + } + + @Override + public int getSize() { + return byteArrayValueKV.getSize(); + } + + @Override + public int calMemory() { + return byteArrayValueKV.calMemory(); + } + + @Override + public Long get(String key) { + ByteBuffer buffer = ByteBuffer.allocate(8); + byte[] bytes = byteArrayValueKV.get(key); + if (bytes == null) { + return null; + } + buffer.put(bytes, 0, 8); + buffer.flip(); + return buffer.getLong(); + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java index 3065a46d..0a6009ae 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/cache/compress/impl/MutilValueKV.java @@ -23,26 +23,25 @@ public abstract class MutilValueKV implements ICacheKV { //按固定大小分割存储 - protected List> valueKVS=new ArrayList<>(); + protected List> valueKVS = new ArrayList<>(); //当前存储的索引 - protected int currentIndex=0; + protected int currentIndex = 0; //每个分片的大小 protected int capacity; - public MutilValueKV(int capacity){ - this.capacity=capacity; + public MutilValueKV(int capacity) { + this.capacity = capacity; } - @Override public T get(String key) { - if(valueKVS==null){ + if (valueKVS == null) { return null; } - for(ICacheKV cacheKV:valueKVS){ - if(cacheKV!=null){ - T value=cacheKV.get(key); - if(value!=null){ + for (ICacheKV cacheKV : valueKVS) { + if (cacheKV != null) { + T value = cacheKV.get(key); + if (value != null) { return value; } } @@ -52,32 +51,32 @@ public T get(String key) { @Override public void put(String key, T value) { - if(valueKVS==null){ + if (valueKVS == null) { return; } - ICacheKV cacheKV= valueKVS.get(currentIndex); - if(cacheKV.getSize()>=capacity){ - synchronized (this){ - cacheKV= valueKVS.get(currentIndex); - if(cacheKV.getSize()>=capacity){ - cacheKV=create(); + ICacheKV cacheKV = valueKVS.get(currentIndex); + if (cacheKV.getSize() >= capacity) { + synchronized (this) { + cacheKV = valueKVS.get(currentIndex); + if (cacheKV.getSize() >= capacity) { + cacheKV = create(); valueKVS.add(cacheKV); currentIndex++; } } } - cacheKV.put(key,value); + cacheKV.put(key, value); } @Override public boolean contains(String key) { - if(valueKVS==null){ + if (valueKVS == null) { return false; } - for(ICacheKV cacheKV:valueKVS){ - if(cacheKV!=null){ - boolean isMatch=cacheKV.contains(key); - if(isMatch){ + for (ICacheKV cacheKV : valueKVS) { + if (cacheKV != null) { + boolean isMatch = cacheKV.contains(key); + if (isMatch) { return true; } } @@ -85,7 +84,6 @@ public boolean contains(String key) { return false; } - protected abstract ICacheKV create(); @Override 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 2119a6bb..211fc17a 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 @@ -25,6 +25,6 @@ public abstract class AbstractSupportShuffleChannelBuilder implements IChannelBu @Override public ISource copy(ISource pipelineSource) { JSONObject jsonObject = JSONObject.parseObject(pipelineSource.toJson()); - return (ISource)ConfigurableUtil.create(pipelineSource.getNameSpace(), pipelineSource.getConfigureName(), jsonObject, pipelineSource.getClass().getName()); + 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/impl/OutputPrintChannel.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/OutputPrintChannel.java index bd7029f7..8c4f63cd 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/OutputPrintChannel.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/impl/OutputPrintChannel.java @@ -17,6 +17,7 @@ package org.apache.rocketmq.streams.common.channel.impl; import java.util.List; + import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.PrintUtil; @@ -26,16 +27,12 @@ */ public class OutputPrintChannel extends AbstractSink { - @Override protected boolean batchInsert(List messages) { for (IMessage msg : messages) { - System.out.println(msg.getMessageValue()); + //System.out.println(msg.getMessageValue()); } return false; } - - - } 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 e56c30a3..82218c2d 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 @@ -17,12 +17,14 @@ package org.apache.rocketmq.streams.common.channel.sink; 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 org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache; @@ -72,13 +74,12 @@ public boolean batchAdd(IMessage fieldName2Value, ISplit split) { } public ISplit getSplit(IMessage message) { - return (ISplit)message.getMessageBody().get(TARGET_QUEUE); + return (ISplit) message.getMessageBody().get(TARGET_QUEUE); } @Override public boolean batchAdd(IMessage fieldName2Value) { messageCache.addCache(fieldName2Value); - return true; } @@ -127,7 +128,7 @@ public boolean batchSave(List messages) { public boolean flush(Set splitIds) { int size = messageCache.flush(splitIds); if (size > 0) { - System.out.println(this.getClass().getSimpleName()+ " finish flush data " + size); + System.out.println(this.getClass().getSimpleName() + " finish flush data " + size); } return size > 0; @@ -135,11 +136,11 @@ public boolean flush(Set splitIds) { @Override public boolean flush(String... splitIds) { - if(splitIds==null){ + if (splitIds == null) { return true; } - Set splitIdSet =new HashSet<>(); - for(String splitId:splitIds){ + Set splitIdSet = new HashSet<>(); + for (String splitId : splitIds) { splitIdSet.add(splitId); } return flush(splitIdSet); @@ -172,16 +173,18 @@ public boolean flushMessage(List messages) { return success; } - @Override public boolean checkpoint(Set splitIds) { + @Override + public boolean checkpoint(Set splitIds) { return flush(splitIds); } - @Override public boolean checkpoint(String... splitIds) { - if(splitIds==null){ + @Override + public boolean checkpoint(String... splitIds) { + if (splitIds == null) { return false; } - Set splitSet=new HashSet<>(); - for(String splitId: splitIds){ + Set splitSet = new HashSet<>(); + for (String splitId : splitIds) { splitSet.add(splitId); } @@ -242,13 +245,13 @@ public IMessageCache getMessageCache() { @Override public Map getFinishedQueueIdAndOffsets(CheckPointMessage checkPointMessage) { - String piplineName = null; - if (IConfigurableIdentification.class.isInstance(checkPointMessage.getStreamOperator())) { - IConfigurableIdentification configurable = (IConfigurableIdentification)checkPointMessage.getStreamOperator(); - piplineName = configurable.getConfigureName(); + String pipelineName = null; + if (checkPointMessage.getStreamOperator() instanceof IConfigurableIdentification) { + IConfigurableIdentification configurable = (IConfigurableIdentification) checkPointMessage.getStreamOperator(); + pipelineName = configurable.getConfigureName(); } SourceState sourceState = this.sourceName2State.get( - CheckPointManager.createSourceName(checkPointMessage.getSource(), piplineName)); + CheckPointManager.createSourceName(checkPointMessage.getSource(), pipelineName)); if (sourceState != null) { return sourceState.getQueueId2Offsets(); } 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 955e9500..4e49b634 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 @@ -150,23 +150,24 @@ public void openSplitModel() { /** * cache filter(regex,like,equals)result */ - private static String FILTER_CACHE_PREPIX="__filter_cache_prefix"; - public void setFilterCache(String expressionStr,String varValue, boolean result){ - this.put(MapKeyUtil.createKey(FILTER_CACHE_PREPIX,expressionStr,varValue),result); + private static String FILTER_CACHE_PREPIX = "__filter_cache_prefix"; + + public void setFilterCache(String expressionStr, String varValue, boolean result) { + this.put(MapKeyUtil.createKey(FILTER_CACHE_PREPIX, expressionStr, varValue), result); } /** * get cache result + * * @param expressionStr * @param varValue * @return */ - public Boolean getFilterCache(String expressionStr,String varValue){ - String key=MapKeyUtil.createKey(FILTER_CACHE_PREPIX,expressionStr,varValue); + public Boolean getFilterCache(String expressionStr, String varValue) { + String key = MapKeyUtil.createKey(FILTER_CACHE_PREPIX, expressionStr, varValue); return (Boolean) this.get(key); } - /** * 获取基于字段缓存的某些值 * @@ -176,7 +177,7 @@ public Boolean getFilterCache(String expressionStr,String varValue){ */ @Deprecated public T getValue(String fieldName) { - return (T)values.get(fieldName); + return (T) values.get(fieldName); } /** @@ -222,7 +223,7 @@ public void setValues(Map values) { } public static List executeScript(IMessage channelMessage, C context, - List> scriptExpressions) { + List> scriptExpressions) { List messages = new ArrayList<>(); if (scriptExpressions == null) { return messages; @@ -356,7 +357,7 @@ protected void copyProperty(AbstractContext context) { context.setSplitModel(this.isSplitModel()); List messages = new ArrayList<>(); for (T tmp : this.getSplitMessages()) { - messages.add(tmp.copy()); + messages.add(tmp.deepCopy()); } context.setSplitMessages(messages); context.monitor = this.monitor; 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 13f13124..244ed3b7 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 @@ -17,13 +17,18 @@ package org.apache.rocketmq.streams.common.topology; import com.alibaba.fastjson.JSONObject; +import com.google.common.collect.Lists; import java.io.Serializable; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.rocketmq.streams.common.cache.compress.impl.LongValueKV; 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.configurable.AbstractConfigurable; import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener; import org.apache.rocketmq.streams.common.configurable.IConfigurableService; @@ -48,6 +53,12 @@ public class ChainPipeline extends Pipeline implements IA private static final long serialVersionUID = -5189371682717444347L; + private final transient int duplicateCacheSize = 1000000; + private transient LongValueKV duplicateCache; + // private transient Map duplicateCache; + private transient List duplicateFields; + private transient int duplicateCacheExpirationTime; + /** * 是否自动启动channel */ @@ -91,6 +102,7 @@ public void setPipelineStatus(Integer pipelineStatus) { /** * 启动一个channel,并给channel应用pipeline */ + public void startChannel() { final String monitorName = createPipelineMonitorName(); if (isInitSuccess()) { @@ -104,8 +116,9 @@ public void startChannel() { pipelineMonitorForChannel = IMonitor.createMonitor(this); } try { - source.start((IStreamOperator)(message, context) -> { + source.start((IStreamOperator) (message, context) -> { //每条消息一个,监控整个链路 + IMonitor pipelineMonitorForStage = context.startMonitor(monitorName); pipelineMonitorForStage.setType(IMonitor.TYPE_DATAPROCESS); message.getHeader().setPiplineName(this.getConfigureName()); @@ -127,10 +140,21 @@ public void startChannel() { } + private String createDuplicateKey(IMessage message) { + List duplicateValues = Lists.newArrayList(); + for (String field : duplicateFields) { + duplicateValues.add(message.getMessageBody().getString(field)); + } + return StringUtil.createMD5Str(String.join("", duplicateValues)); + } + private String createPipelineMonitorName() { return MapKeyUtil.createKeyBySign(".", getType(), getNameSpace(), getConfigureName()); } + private static AtomicInteger total = new AtomicInteger(0); + private static AtomicInteger hitCache = new AtomicInteger(0); + /** * 可以替换某个阶段的阶段,而不用配置的阶段 * @@ -141,6 +165,26 @@ private String createPipelineMonitorName() { */ @Override protected T doMessageInner(T t, AbstractContext context, AbstractStage... replaceStage) { + if (this.duplicateCache != null && this.duplicateFields != null && !this.duplicateFields.isEmpty() && !t.getHeader().isSystemMessage()) { + total.incrementAndGet(); + String duplicateKey = createDuplicateKey(t); + Long cacheTime = this.duplicateCache.get(duplicateKey); + Long currentTime = System.currentTimeMillis(); + if (cacheTime != null && currentTime - cacheTime < this.duplicateCacheExpirationTime) { + hitCache.incrementAndGet(); + context.breakExecute(); + return t; + } else { + this.duplicateCache.put(duplicateKey, currentTime); + if (this.duplicateCache.getSize() > duplicateCacheSize) { + this.duplicateCache = new LongValueKV(this.duplicateCacheSize); + } + } + if (total.get() % 5000 == 0) { + System.out.printf("total: %s, hit: %s%n", total.get(), hitCache.get()); + } + } + if (!t.getHeader().isSystemMessage()) { MessageGloableTrace.joinMessage(t);//关联全局监控器 } @@ -164,7 +208,8 @@ public boolean isTopology() { return isTopology(this.channelNextStageLabel); } - public void doNextStages(AbstractContext context, String msgPrewSourceName, List nextStageLabel, String prewSQLNodeName, AbstractStage... replaceStage) { + public void doNextStages(AbstractContext context, String msgPrewSourceName, List nextStageLabel, + String prewSQLNodeName, AbstractStage... replaceStage) { if (!isTopology(nextStageLabel)) { return; @@ -178,7 +223,7 @@ public void doNextStages(AbstractContext context, String msgPrewSourceName, List if (size > 1) { copyContext = context.copy(); } - T msg = (T)copyContext.getMessage(); + T msg = (T) copyContext.getMessage(); AbstractStage oriStage = stageMap.get(lable); if (oriStage == null) { if (stages != null && stages.size() > 0) { @@ -224,7 +269,7 @@ public void doNextStages(AbstractContext context, String msgPrewSourceName, List continue; } else { if (ChainStage.class.isInstance(stage)) { - ChainStage chainStage = (ChainStage)stage; + ChainStage chainStage = (ChainStage) stage; String msgSourceName = chainStage.getMsgSourceName(); if (StringUtil.isNotEmpty(msgSourceName)) { msgPrewSourceName = msgSourceName; @@ -286,7 +331,7 @@ protected JSONObject createStageInfo(AbstractStage stage) { JSONObject jsonObject = null; if (stage instanceof ChainStage) { jsonObject = new JSONObject(); - ChainStage chainStage = (ChainStage)stage; + ChainStage chainStage = (ChainStage) stage; return chainStage.toJsonObject(); //String entityName = chainStage.getEntityName(); ////todo 需要改写 @@ -324,18 +369,15 @@ public void setSource(ISource source) { public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { for (AbstractStage stage : getStages()) { stage.setPipeline(this); - if (IAfterConfigurableRefreshListener.class.isInstance(stage)) { - if (AbstractConfigurable.class.isInstance(stage)) { - AbstractConfigurable abstractConfigurable = (AbstractConfigurable)stage; - if (abstractConfigurable.isInitSuccess() == false && this.isInitSuccess() == false) { - this.setInitSuccess(false); - return; - } + if (stage instanceof IAfterConfigurableRefreshListener) { + if (!stage.isInitSuccess() && !this.isInitSuccess()) { + this.setInitSuccess(false); + return; } - IAfterConfigurableRefreshListener afterConfiguableRefreshListerner = - (IAfterConfigurableRefreshListener)stage; + IAfterConfigurableRefreshListener afterConfigurableRefreshListener = + (IAfterConfigurableRefreshListener) stage; - afterConfiguableRefreshListerner.doProcessAfterRefreshConfigurable(configurableService); + afterConfigurableRefreshListener.doProcessAfterRefreshConfigurable(configurableService); } } @@ -349,9 +391,9 @@ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableS startChannel(); } this.source = source; - if (AbstractConfigurable.class.isInstance(source)) { - AbstractConfigurable abstractConfigurable = (AbstractConfigurable)source; - if (abstractConfigurable.isInitSuccess() == false && this.isInitSuccess()) { + if (source instanceof AbstractConfigurable) { + AbstractConfigurable abstractConfigurable = (AbstractConfigurable) source; + if (!abstractConfigurable.isInitSuccess() && this.isInitSuccess()) { this.setInitSuccess(false); return; } @@ -361,6 +403,23 @@ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableS if ((isAutoStart || isPublish()) && isInitSuccess()) { startChannel(); } + + //增加去重的逻辑 + String duplicateFieldNameStr = ComponentCreator.getProperties().getProperty(getConfigureName() + ".duplicate.fields.names"); + if (duplicateFieldNameStr != null && !duplicateFieldNameStr.isEmpty()) { + this.duplicateFields = Lists.newArrayList(); + this.duplicateFields.addAll(Arrays.asList(duplicateFieldNameStr.split(";"))); + } + if (this.duplicateCache == null && this.duplicateFields != null) { + this.duplicateCache = new LongValueKV(this.duplicateCacheSize); + } + String duplicateCacheExpirationStr = ComponentCreator.getProperties().getProperty(getConfigureName() + ".duplicate.expiration.time"); + if (duplicateCacheExpirationStr != null && !duplicateCacheExpirationStr.isEmpty()) { + this.duplicateCacheExpirationTime = Integer.parseInt(duplicateCacheExpirationStr); + } else { + this.duplicateCacheExpirationTime = 86400000; + } + } public Map createStageMap() { 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 4dff54f5..5cbac0f8 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 @@ -80,14 +80,14 @@ protected IMessage doProcess(IMessage message, AbstractContext context) { /** * 主要是输出可能影响线上数据,可以通过配置文件的开关,把所有的输出,都指定到一个其他输出中 */ - if(openMockChannel()){ - if(mockSink!=null){ + if (openMockChannel()) { + if (mockSink != null) { mockSink.batchAdd(message.deepCopy()); return message; } return message; } - sink.batchAdd(message); + sink.batchAdd(message.deepCopy()); return message; } @@ -100,23 +100,23 @@ public String getName() { @Override public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) { - ISink realSink=null; - if(openMockChannel()&&mockSink!=null){ - realSink=mockSink; - }else { - realSink=sink; + ISink realSink = null; + if (openMockChannel() && mockSink != null) { + realSink = mockSink; + } else { + realSink = sink; } - if(message.getHeader().isNeedFlush()){ - Set queueIds=new HashSet<>(); - if(message.getHeader().getCheckpointQueueIds()!=null){ + if (message.getHeader().isNeedFlush()) { + Set queueIds = new HashSet<>(); + if (message.getHeader().getCheckpointQueueIds() != null) { queueIds.addAll(message.getHeader().getCheckpointQueueIds()); } - if(StringUtil.isNotEmpty(message.getHeader().getQueueId())){ + if (StringUtil.isNotEmpty(message.getHeader().getQueueId())) { queueIds.add(message.getHeader().getQueueId()); } realSink.checkpoint(queueIds); } - CheckPointState checkPointState= new CheckPointState(); + CheckPointState checkPointState = new CheckPointState(); checkPointState.setQueueIdAndOffset(realSink.getFinishedQueueIdAndOffsets(checkPointMessage)); checkPointMessage.reply(checkPointState); @@ -182,33 +182,33 @@ public ISink getSink() { @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { - sink=configurableService.queryConfigurable(ISink.TYPE, sinkName); - if(sink==null){ + sink = configurableService.queryConfigurable(ISink.TYPE, sinkName); + if (sink == null) { sink = configurableService.queryConfigurable(IChannel.TYPE, sinkName); } metaData = configurableService.queryConfigurable(MetaData.TYPE, metaDataName); - mockSink=getMockChannel(configurableService,sink.getNameSpace()); + mockSink = getMockChannel(configurableService, sink.getNameSpace()); } - private ISink getMockChannel(IConfigurableService configurableService,String nameSpace) { - String type=ComponentCreator.getProperties().getProperty("out.mock.type"); - if(type==null){ + private ISink getMockChannel(IConfigurableService configurableService, String nameSpace) { + String type = ComponentCreator.getProperties().getProperty("out.mock.type"); + if (type == null) { return null; } - ISink mockSink= configurableService.queryConfigurable(ISink.TYPE,OUT_MOCK_SWITCH+"_"+type); - if(mockSink==null){ - mockSink= configurableService.queryConfigurable(IChannel.TYPE,OUT_MOCK_SWITCH+"_"+type); + ISink mockSink = configurableService.queryConfigurable(ISink.TYPE, OUT_MOCK_SWITCH + "_" + type); + if (mockSink == null) { + mockSink = configurableService.queryConfigurable(IChannel.TYPE, OUT_MOCK_SWITCH + "_" + type); } return mockSink; } - protected boolean openMockChannel(){ - String swtich=ComponentCreator.getProperties().getProperty(OUT_MOCK_SWITCH); - if(swtich==null){ + protected boolean openMockChannel() { + String swtich = ComponentCreator.getProperties().getProperty(OUT_MOCK_SWITCH); + if (swtich == null) { return false; } - if("true".equals(swtich)){ + if ("true".equals(swtich)) { return true; } return false; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java index 9f8f46d3..fe8a3bbb 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java @@ -34,7 +34,8 @@ public class SQLUtil { private static final String INSERT_IGNORE = "INSERT IGNORE INTO"; private static final String REPLACE = "REPLACE INTO"; - public static String createReplacesInsertSql(MetaData metaData, Map fieldName2Value, Boolean containsIdField) { + public static String createReplacesInsertSql(MetaData metaData, Map fieldName2Value, + Boolean containsIdField) { String insertSQL = createInsertSql(metaData, fieldName2Value, containsIdField); insertSQL = insertSQL.replaceFirst(INSERT, REPLACE); return insertSQL; @@ -60,7 +61,8 @@ public static String createDuplicateKeyUpdateSQL(MetaData metaData) { return stringBuilder.toString(); } - public static String createIgnoreInsertSql(MetaData metaData, Map fieldName2Value, Boolean containsIdField) { + public static String createIgnoreInsertSql(MetaData metaData, Map fieldName2Value, + Boolean containsIdField) { String insertSQL = createInsertSql(metaData, fieldName2Value, containsIdField); insertSQL = insertSQL.replaceFirst(INSERT, INSERT_IGNORE); return insertSQL; @@ -70,7 +72,8 @@ public static String createInsertSql(MetaData metaData, Map fiel return createInsertSql(metaData, fieldName2Value, null); } - public static String createInsertSql(MetaData metaData, Map fieldName2Value, Boolean containsIdField) { + public static String createInsertSql(MetaData metaData, Map fieldName2Value, + Boolean containsIdField) { StringBuilder sql = new StringBuilder(INSERT + " " + metaData.getTableName() + "("); StringBuilder fieldSql = new StringBuilder(); @@ -106,7 +109,8 @@ public static String createInsertValuesSQL(MetaData metaData, List fieldName2Value, StringBuilder fieldSql, StringBuilder valueSql) { + protected static String createInsertValuesSQL(MetaData metaData, Map fieldName2Value, + StringBuilder fieldSql, StringBuilder valueSql) { boolean isIncrement = true; if (fieldName2Value.containsKey(metaData.getIdFieldName())) { isIncrement = false; @@ -114,7 +118,8 @@ protected static String createInsertValuesSQL(MetaData metaData, Map fieldName2Value, StringBuilder fieldSql, StringBuilder valueSql, boolean containsIdField) { + protected static String createInsertValuesSQL(MetaData metaData, Map fieldName2Value, + StringBuilder fieldSql, StringBuilder valueSql, boolean containsIdField) { boolean isFirst = true; valueSql.append("("); //if (fieldName2Value.containsKey(metaData.getIdFieldName())) { @@ -172,16 +177,16 @@ public static String getFieldSqlValue(MetaData metaData, String filedName, Objec // } String result = null; if (DataTypeUtil.isDate(field.getDataType().getDataClass())) { - result = DateUtil.format((Date)value); + result = DateUtil.format((Date) value); } else if (JSONObject.class.isInstance(value)) { - result = ((JSONObject)value).toJSONString(); + result = ((JSONObject) value).toJSONString(); } else { result = value.toString(); } return "'" + handleSpecialCharInSql(result) + "'"; } else { if (DataTypeUtil.isBoolean(field.getDataType().getDataClass())) { - boolean boolValue = (Boolean)value; + boolean boolValue = (Boolean) value; return boolValue ? "1" : "0"; } return value + ""; @@ -261,12 +266,12 @@ public static String parseIbatisSQL(Object object, String ibatisSQL, boolean con if (object == null) { return ibatisSQL; } - if (object == null || StringUtil.isEmpty(ibatisSQL)) { + if (StringUtil.isEmpty(ibatisSQL)) { return null; } List vars = parseIbatisSQLVars(ibatisSQL); - if (vars == null || vars.size() == 0) { + if (vars.size() == 0) { return ibatisSQL; } String sql = ibatisSQL; @@ -274,19 +279,22 @@ public static String parseIbatisSQL(Object object, String ibatisSQL, boolean con Object value = getBeanFieldValue(object, varName); String valueSQL = null; - if (value != null & !String.class.isInstance(value) && !Date.class.isInstance(value)) { + if (value != null & !(value instanceof String) && !Date.class.isInstance(value)) { valueSQL = value.toString(); } - if (value != null && String.class.isInstance(value)) { + if (value instanceof String) { value = value.toString().replace("'", "''"); + if (value.toString().contains("\\")) { + value = value.toString().replaceAll("\\\\", "\\\\\\\\"); + } if (containsQuotation && (ibatisSQL.indexOf("'#{" + varName + "}'") > -1 || ibatisSQL.indexOf("`#{" + varName + "}`") > -1)) { valueSQL = value + ""; } else { valueSQL = "'" + value + "'"; } } - if (value != null && Date.class.isInstance(value)) { - String valueDate = DateUtil.format((Date)value); + if (value instanceof Date) { + String valueDate = DateUtil.format((Date) value); if (containsQuotation && ibatisSQL.indexOf("'#{" + varName + "}'") > -1) { valueSQL = valueDate; } else { @@ -314,20 +322,20 @@ public static String parseIbatisSQL(Object object, String ibatisSQL, boolean con } protected static Object getBeanFieldValue(Object object, String varName) { - if (JSONObject.class.isInstance(object)) { - JSONObject jsonObject = (JSONObject)object; + if (object instanceof JSONObject) { + JSONObject jsonObject = (JSONObject) object; return jsonObject.get(varName); - } else if (Map.class.isInstance(object)) { - Map paras = (Map)object; + } else if (object instanceof Map) { + Map paras = (Map) object; return paras.get(varName); } else { - if (IConfigurable.class.isInstance(object) && varName.equals(IConfigurable.JSON_PROPERTY)) { - IConfigurable configurable = (IConfigurable)object; + if (object instanceof IConfigurable && varName.equals(IConfigurable.JSON_PROPERTY)) { + IConfigurable configurable = (IConfigurable) object; return configurable.toJson(); } - if (BasedConfigurable.class.isInstance(object) && varName.equals(IConfigurable.STATUS_PROPERTY)) { - BasedConfigurable basedConfigurable = (BasedConfigurable)object; + if (object instanceof BasedConfigurable && varName.equals(IConfigurable.STATUS_PROPERTY)) { + BasedConfigurable basedConfigurable = (BasedConfigurable) object; return basedConfigurable.getStatus(); } return ReflectUtil.getBeanFieldValue(object, varName); @@ -406,7 +414,7 @@ public static String createInSql(boolean isString, String... values) { if (!isString) { stringBuilder.append(value); } else { - stringBuilder.append("'" + value + "'"); + StringBuilder append = stringBuilder.append("'" + value + "'"); } } 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 896e6213..25c5bee4 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 @@ -17,6 +17,7 @@ package org.apache.rocketmq.streams.configurable.service; import com.alibaba.fastjson.JSONObject; + import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -27,6 +28,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.component.AbstractComponent; @@ -133,9 +135,7 @@ public boolean refreshConfigurable(String namespace) { if (configures != null && configures.isQuerySuccess() && configures.getConfigurables() != null) { // List configureList = filterConfigure(configures.getConfigure()); List configurables = configures.getConfigurables(); - List configurableList = checkAndUpdateConfigurables(namespace, configurables, - tempType2ConfigurableMap, tempName2ConfigurableMap, - configures.getConfigurables()); + List configurableList = checkAndUpdateConfigurables(namespace, configurables, tempType2ConfigurableMap, tempName2ConfigurableMap, configures.getConfigurables()); // this.namespace2ConfigurableMap = namespace2ConfigurableMap; for (IConfigurable configurable : configurableList) { if (configurable instanceof IAfterConfigurableRefreshListener) { 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 9080479d..e51e94fd 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 @@ -317,7 +317,7 @@ public IConfigurableService getConfigurableService() { @Override public AbstractContext copy() { - IMessage message = this.message.copy(); + IMessage message = this.message.deepCopy(); RuleContext context = new RuleContext(nameSpace, message.getMessageBody(), rule, contextConfigure); super.copyProperty(context); context.actionExecutor = actionExecutor; diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/storages/DBLeaseStorage.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/storages/DBLeaseStorage.java index bd7ef82e..c5a0c27b 100644 --- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/storages/DBLeaseStorage.java +++ b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/storages/DBLeaseStorage.java @@ -21,6 +21,7 @@ import java.util.Date; import java.util.List; import java.util.Map; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.utils.DateUtil; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/context/FunctionContext.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/context/FunctionContext.java index 2f2ccf12..3255a693 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/context/FunctionContext.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/context/FunctionContext.java @@ -48,7 +48,7 @@ public T getMessage() { @Override public AbstractContext copy() { - IMessage message = this.message.copy(); + IMessage message = this.message.deepCopy(); FunctionContext context = new FunctionContext(message); super.copyProperty(context); context.setFunctionService(this.functionService); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/FieldFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/FieldFunction.java index 2541a82b..6d375be8 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/FieldFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/field/FieldFunction.java @@ -16,6 +16,8 @@ */ package org.apache.rocketmq.streams.script.function.impl.field; +import com.google.common.collect.Lists; +import java.util.List; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.script.annotation.Function; @@ -29,17 +31,17 @@ public class FieldFunction { @FunctionMethod(value = "field", alias = "get", comment = "获取字段值") public T getFieldValue(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "字段的名称,不需要引号") String fieldName) { + @FunctionParamter(value = "string", comment = "字段的名称,不需要引号") String fieldName) { String name = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(name)) { name = fieldName; } - return (T)message.getMessageBody().get(name); + return (T) message.getMessageBody().get(name); } @FunctionMethod(value = "char_length", alias = "len", comment = "求字段代码字符串或常量的长度") public int len(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName) { String value = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(value)) { return 0; @@ -49,7 +51,7 @@ public int len(IMessage message, FunctionContext context, @FunctionMethod(value = "lower", alias = "low", comment = "把字符串转换称小写") public String lower(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String fieldName) { String value = FunctionUtils.getValueString(message, context, fieldName); if (StringUtil.isEmpty(value)) { return null; @@ -59,7 +61,7 @@ public String lower(IMessage message, FunctionContext context, @FunctionMethod(value = "concat", comment = "连接字符串") public String concat(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String... fieldNames) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String... fieldNames) { StringBuilder sb = new StringBuilder(); for (String fieldName : fieldNames) { String value = FunctionUtils.getValueString(message, context, fieldName); @@ -70,28 +72,30 @@ public String concat(IMessage message, FunctionContext context, @FunctionMethod(value = "concat_ws", alias = "concat_sign", comment = "通过分隔符把字符串拼接在一起") public String concat_ws(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表分隔符的字段名或常量") String sign, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String... fieldNames) { - StringBuilder sb = new StringBuilder(); - boolean isFirst = true; + @FunctionParamter(value = "string", comment = "代表分隔符的字段名或常量") String sign, + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String... fieldNames) { sign = FunctionUtils.getValueString(message, context, sign); + if (sign == null) { + sign = ","; + } + List values = Lists.newArrayList(); for (String fieldName : fieldNames) { - if (isFirst) { - isFirst = false; - } else { - sb.append(sign); - } String value = FunctionUtils.getValueString(message, context, fieldName); - sb.append(value); + if (value != null) { + values.add(value); + } } - return sb.toString(); + if (values.isEmpty()) { + return null; + } + return String.join(sign, values); } @FunctionMethod(value = "lpad", comment = "在原串左边补n个pad字符串,如果原串长度小于len,则截断,使得整个字符串长度为len") public String lpad(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String ori, - @FunctionParamter(value = "string", comment = "代表字符串长度字段名,数字或常量") String lenStr, - @FunctionParamter(value = "string", comment = "代表补齐字符串的字段名或常量") String pad) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String ori, + @FunctionParamter(value = "string", comment = "代表字符串长度字段名,数字或常量") String lenStr, + @FunctionParamter(value = "string", comment = "代表补齐字符串的字段名或常量") String pad) { if (StringUtil.isEmpty(ori) || pad == null) { return null; } @@ -121,9 +125,9 @@ public String lpad(IMessage message, FunctionContext context, @FunctionMethod(value = "rpad", comment = "在原串左边补n个pad字符串,如果原串长度小于len,则截断,使得整个字符串长度为len") public String rpad(IMessage message, FunctionContext context, - @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String ori, - @FunctionParamter(value = "string", comment = "代表字符串长度字段名,数字或常量") String lenStr, - @FunctionParamter(value = "string", comment = "代表补齐字符串的字段名或常量") String pad) { + @FunctionParamter(value = "string", comment = "代表字符串的字段名或常量") String ori, + @FunctionParamter(value = "string", comment = "代表字符串长度字段名,数字或常量") String lenStr, + @FunctionParamter(value = "string", comment = "代表补齐字符串的字段名或常量") String pad) { if (StringUtil.isEmpty(ori) || pad == null) { return null; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/flatmap/SplitArrayFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/flatmap/SplitArrayFunction.java index f0766960..85f5727d 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/flatmap/SplitArrayFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/flatmap/SplitArrayFunction.java @@ -18,8 +18,10 @@ import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; + import java.util.List; import java.util.Map; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.rocketmq.streams.common.context.IMessage; @@ -132,7 +134,7 @@ public void splitA(IMessage channelMessage, FunctionContext context, context.openSplitModel(); for (int i = 0; i < values.length; i++) { String value = values[i]; - if("null".equals(value.toLowerCase())){ + if ("null".equalsIgnoreCase(value)) { continue; } IMessage newMessage = channelMessage.deepCopy(); diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/AbstractScriptProxy.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/AbstractScriptProxy.java index 6bdfc5a1..5071a022 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/AbstractScriptProxy.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/AbstractScriptProxy.java @@ -29,27 +29,26 @@ public abstract class AbstractScriptProxy implements IScriptExpression { protected IScriptExpression origExpression; + public AbstractScriptProxy(IScriptExpression origExpression) { - this.origExpression=origExpression; + this.origExpression = origExpression; } public abstract List getCacheFilters(); + public abstract boolean supportOptimization(IScriptExpression scriptExpression); - - public abstract boolean supportOptimization(IScriptExpression scriptExpression) ; protected String getParameterValue(IScriptParamter scriptParamter) { if (!ScriptParameter.class.isInstance(scriptParamter)) { return null; } - ScriptParameter parameter = (ScriptParameter)scriptParamter; + ScriptParameter parameter = (ScriptParameter) scriptParamter; if (parameter.getRigthVarName() != null) { return null; } return FunctionUtils.getConstant(parameter.getLeftVarName()); } - @Override public List getScriptParamters() { return this.origExpression.getScriptParamters(); } @@ -63,7 +62,7 @@ protected String getParameterValue(IScriptParamter scriptParamter) { } @Override public Object getScriptParamter(IMessage message, FunctionContext context) { - return this.origExpression.getScriptParamter(message,context); + return this.origExpression.getScriptParamter(message, context); } public void setOrigExpression(IScriptExpression origExpression) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/CaseScriptExpressionProxy.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/CaseScriptExpressionProxy.java index 6f6140c4..4b507c6a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/CaseScriptExpressionProxy.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/CaseScriptExpressionProxy.java @@ -31,39 +31,40 @@ public CaseScriptExpressionProxy(IScriptExpression origExpression) { } @Override public List getCacheFilters() { - List result=new ArrayList<>(); - GroupScriptExpression groupScriptExpression=(GroupScriptExpression)this.origExpression; - recursion(groupScriptExpression,result); + List result = new ArrayList<>(); + GroupScriptExpression groupScriptExpression = (GroupScriptExpression) this.origExpression; + recursion(groupScriptExpression, result); return result; } /** * recursion else if GroupScriptExpression list + * * @param groupScriptExpression * @param cacheFilters */ - protected void recursion(GroupScriptExpression groupScriptExpression,List cacheFilters){ - IScriptExpression scriptExpression= groupScriptExpression.getIfExpresssion(); - AbstractScriptProxy abstractExpressionProxy= ScriptProxyFactory.getInstance().create(scriptExpression); - if(abstractExpressionProxy!=null){ + protected void recursion(GroupScriptExpression groupScriptExpression, List cacheFilters) { + IScriptExpression scriptExpression = groupScriptExpression.getIfExpresssion(); + AbstractScriptProxy abstractExpressionProxy = ScriptProxyFactory.getInstance().create(scriptExpression); + if (abstractExpressionProxy != null) { groupScriptExpression.setIfExpresssion(abstractExpressionProxy); cacheFilters.addAll(abstractExpressionProxy.getCacheFilters()); } - if(groupScriptExpression.getElseIfExpressions()!=null){ - for(GroupScriptExpression expression:groupScriptExpression.getElseIfExpressions()){ - recursion(expression,cacheFilters); + if (groupScriptExpression.getElseIfExpressions() != null) { + for (GroupScriptExpression expression : groupScriptExpression.getElseIfExpressions()) { + recursion(expression, cacheFilters); } } } @Override public boolean supportOptimization(IScriptExpression scriptExpression) { - if(scriptExpression instanceof GroupScriptExpression){ - return true; - } - return false; + if (scriptExpression instanceof GroupScriptExpression) { + return true; + } + return false; } @Override public Object executeExpression(IMessage message, FunctionContext context) { - return this.origExpression.executeExpression(message,context); + return this.origExpression.executeExpression(message, context); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/EqualsProxy.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/EqualsProxy.java index ba1d06d5..5f9aca91 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/EqualsProxy.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/EqualsProxy.java @@ -17,10 +17,7 @@ package org.apache.rocketmq.streams.script.optimization.performance; -import org.apache.rocketmq.streams.common.optimization.cachefilter.ICacheFilter; -import org.apache.rocketmq.streams.common.optimization.cachefilter.ICacheFilterBulider; import org.apache.rocketmq.streams.script.function.impl.condition.EqualsFunction; -import org.apache.rocketmq.streams.script.function.impl.string.RegexFunction; import org.apache.rocketmq.streams.script.service.IScriptExpression; import org.apache.rocketmq.streams.script.service.IScriptParamter; @@ -40,7 +37,7 @@ public boolean supportOptimization(IScriptExpression scriptExpression) { } @Override protected String getVarName() { - return getParameterValue((IScriptParamter)this.origExpression.getScriptParamters().get(0)); + return getParameterValue((IScriptParamter) this.origExpression.getScriptParamters().get(0)); } } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/RegexProxy.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/RegexProxy.java index d3cc1b24..30c214a6 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/RegexProxy.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/RegexProxy.java @@ -17,14 +17,11 @@ package org.apache.rocketmq.streams.script.optimization.performance; -import org.apache.rocketmq.streams.common.optimization.cachefilter.ICacheFilter; -import org.apache.rocketmq.streams.common.optimization.cachefilter.ICacheFilterBulider; -import org.apache.rocketmq.streams.script.function.impl.condition.EqualsFunction; import org.apache.rocketmq.streams.script.function.impl.string.RegexFunction; import org.apache.rocketmq.streams.script.service.IScriptExpression; import org.apache.rocketmq.streams.script.service.IScriptParamter; -public class RegexProxy extends SimpleScriptExpressionProxy { +public class RegexProxy extends SimpleScriptExpressionProxy { public RegexProxy(IScriptExpression origExpression) { super(origExpression); } @@ -40,8 +37,7 @@ public boolean supportOptimization(IScriptExpression scriptExpression) { } @Override protected String getVarName() { - return getParameterValue((IScriptParamter)this.origExpression.getScriptParamters().get(0)); + return getParameterValue((IScriptParamter) this.origExpression.getScriptParamters().get(0)); } - } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptExpressionGroupsProxy.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptExpressionGroupsProxy.java index 220350c3..5433e609 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptExpressionGroupsProxy.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptExpressionGroupsProxy.java @@ -30,28 +30,31 @@ import org.apache.rocketmq.streams.script.service.IScriptParamter; public class ScriptExpressionGroupsProxy extends CacheFilterManager implements IScriptExpression { - protected List scriptExpressions=new ArrayList<>(); + protected List scriptExpressions = new ArrayList<>(); public ScriptExpressionGroupsProxy(int elementCount, int capacity) { super(elementCount, capacity); } + public void removeLessCount() { - Map newFilterOptimizationMap=new HashMap<>(); - for(String varName:this.filterOptimizationMap.keySet()){ - CacheFilterGroup cacheFilterGroup =this.filterOptimizationMap.get(varName); - if(cacheFilterGroup.getSize()>5){ - newFilterOptimizationMap.put(varName,cacheFilterGroup); + Map newFilterOptimizationMap = new HashMap<>(); + for (String varName : this.filterOptimizationMap.keySet()) { + CacheFilterGroup cacheFilterGroup = this.filterOptimizationMap.get(varName); + if (cacheFilterGroup.getSize() > 5) { + newFilterOptimizationMap.put(varName, cacheFilterGroup); } } - this.filterOptimizationMap=newFilterOptimizationMap; + this.filterOptimizationMap = newFilterOptimizationMap; } - public void addScriptExpression(IScriptExpression scriptExpression){ + + public void addScriptExpression(IScriptExpression scriptExpression) { this.scriptExpressions.add(scriptExpression); } + @Override public Object executeExpression(IMessage message, FunctionContext context) { - this.execute(message,context); - for(IScriptExpression scriptExpression:scriptExpressions){ - scriptExpression.executeExpression(message,context); + this.execute(message, context); + for (IScriptExpression scriptExpression : scriptExpressions) { + scriptExpression.executeExpression(message, context); } return null; } @@ -84,5 +87,4 @@ public void addScriptExpression(IScriptExpression scriptExpression){ return null; } - } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptOptimization.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptOptimization.java index ae147ee4..9d43e2b6 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptOptimization.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptOptimization.java @@ -16,41 +16,33 @@ */ package org.apache.rocketmq.streams.script.optimization.performance; +import org.apache.rocketmq.streams.common.optimization.cachefilter.ICacheFilter; +import org.apache.rocketmq.streams.script.operator.expression.GroupScriptExpression; +import org.apache.rocketmq.streams.script.operator.impl.FunctionScript; +import org.apache.rocketmq.streams.script.service.IScriptExpression; + 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.Map.Entry; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.optimization.HyperscanRegex; -import org.apache.rocketmq.streams.common.optimization.cachefilter.ICacheFilter; -import org.apache.rocketmq.streams.script.context.FunctionContext; -import org.apache.rocketmq.streams.script.operator.expression.GroupScriptExpression; -import org.apache.rocketmq.streams.script.operator.expression.ScriptExpression; -import org.apache.rocketmq.streams.script.operator.impl.FunctionScript; -import org.apache.rocketmq.streams.script.service.IScriptExpression; public class ScriptOptimization { protected String name;//function script namespace,name - /** * optimizate expression */ - protected ScriptExpressionGroupsProxy scriptExpressionGroupsProxy =new ScriptExpressionGroupsProxy(160,1000000); + protected ScriptExpressionGroupsProxy scriptExpressionGroupsProxy = new ScriptExpressionGroupsProxy(160, 1000000); //the optimizated script protected List scriptExpressions; - //newFieldName created in the script protected Map newFieldName2Expressions = new HashMap<>(); - /** * Optimization once */ @@ -61,8 +53,8 @@ public class ScriptOptimization { * * @param scriptExpressions */ - public ScriptOptimization(String name,List scriptExpressions) { - this.name=name; + public ScriptOptimization(String name, List scriptExpressions) { + this.name = name; this.scriptExpressions = scriptExpressions; /** @@ -91,8 +83,6 @@ public boolean supportOptimize() { return false; } - - /** * 把表达式拆成3段,创建变量的,正则类,其他。正则类用HyperscanRegex做优化 */ @@ -100,7 +90,7 @@ public List optimize() { if (!startOptimization.compareAndSet(false, true)) { return this.scriptExpressions; } - Set newVarNames=new HashSet<>(); + Set newVarNames = new HashSet<>(); List allScriptExpressions = new ArrayList<>();//最终输出的表达式列表 List proxyExpressions = new ArrayList<>();//最后执行的脚本,在执行完正则后执行的部分 List lastExpressions = new ArrayList<>();//最后执行的脚本,在执行完正则后执行的部分 @@ -108,46 +98,39 @@ public List optimize() { for (IScriptExpression scriptExpression : scriptExpressions) { Set newFieldNames = scriptExpression.getNewFieldNames(); - if (newFieldNames != null&&newFieldNames.size() > 0) { + if (newFieldNames != null && newFieldNames.size() > 0) { String newFieldName = newFieldNames.iterator().next(); newVarNames.add(newFieldName); } - - - - - IScriptExpression scriptExpressionProxy = createProxy(scriptExpression,newVarNames); + IScriptExpression scriptExpressionProxy = createProxy(scriptExpression, newVarNames); String functionName = scriptExpressionProxy.getFunctionName(); - if(scriptExpressionProxy instanceof AbstractScriptProxy){ + if (scriptExpressionProxy instanceof AbstractScriptProxy) { proxyExpressions.add(scriptExpressionProxy); - }else if("trim".equals(functionName) || "lower".equals(functionName) || "concat".equals(functionName)){ + } else if ("trim".equals(functionName) || "lower".equals(functionName) || "concat".equals(functionName)) { mapExpressions.add(scriptExpressionProxy); - }else { + } else { lastExpressions.add(scriptExpressionProxy); } } allScriptExpressions.addAll(mapExpressions);//把优先执行的表达式添加上 - if(this.scriptExpressionGroupsProxy.scriptExpressions.size()>0){ + if (this.scriptExpressionGroupsProxy.scriptExpressions.size() > 0) { allScriptExpressions.add(this.scriptExpressionGroupsProxy); } allScriptExpressions.addAll(lastExpressions);//把剩余的表达式增加到list中 - this.scriptExpressions=allScriptExpressions; + this.scriptExpressions = allScriptExpressions; this.scriptExpressionGroupsProxy.removeLessCount(); return this.scriptExpressions; } - - - /** * 如果脚本中有较多的正则表达式,则统一注册到正则库,并行执行。 * * @param scriptExpression * @return */ - protected IScriptExpression createProxy(IScriptExpression scriptExpression,Set newVarNames) { + protected IScriptExpression createProxy(IScriptExpression scriptExpression, Set newVarNames) { AbstractScriptProxy scriptProxy = ScriptProxyFactory.getInstance().create(scriptExpression); if (scriptProxy == null) { return scriptExpression; @@ -158,22 +141,21 @@ protected IScriptExpression createProxy(IScriptExpression scriptExpression,Set cacheFilters=scriptProxy.getCacheFilters(); - if(cacheFilters!=null){ - for(ICacheFilter cacheFilter:cacheFilters){ - this.scriptExpressionGroupsProxy.addOptimizationExpression(this.name,cacheFilter); + List cacheFilters = scriptProxy.getCacheFilters(); + if (cacheFilters != null) { + for (ICacheFilter cacheFilter : cacheFilters) { + this.scriptExpressionGroupsProxy.addOptimizationExpression(this.name, cacheFilter); } } return scriptProxy; } - public static void main(String[] args) { String scriptValue = "source='netstat_ob';\n" + "____regex_10001=regex(std_cmdline,'^(((/?([a-zA-Z0-9_\\.\\-]+/){1,20})bin/)|/bin/|/|-)?" diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptProxyFactory.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptProxyFactory.java index 514d6dd2..5118c1da 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptProxyFactory.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/ScriptProxyFactory.java @@ -26,33 +26,31 @@ import org.apache.rocketmq.streams.script.service.IScriptExpression; public class ScriptProxyFactory { - protected List expressionProxies=new ArrayList<>(); - protected static ScriptProxyFactory expressionProxyFactory=new ScriptProxyFactory(); - protected static AtomicBoolean isFinishScan=new AtomicBoolean(false); - protected AbstractScan scan=new AbstractScan() { + protected List expressionProxies = new ArrayList<>(); + protected static ScriptProxyFactory expressionProxyFactory = new ScriptProxyFactory(); + protected static AtomicBoolean isFinishScan = new AtomicBoolean(false); + protected AbstractScan scan = new AbstractScan() { @Override protected void doProcessor(Class clazz) { - if(AbstractScriptProxy.class.isAssignableFrom(clazz)&&!Modifier.isAbstract(clazz.getModifiers())){ - AbstractScriptProxy abstractExpressionProxy=(AbstractScriptProxy)ReflectUtil.forInstance(clazz,new Class[]{IScriptExpression.class},new Object[]{null}); + if (AbstractScriptProxy.class.isAssignableFrom(clazz) && !Modifier.isAbstract(clazz.getModifiers())) { + AbstractScriptProxy abstractExpressionProxy = (AbstractScriptProxy) ReflectUtil.forInstance(clazz, new Class[] {IScriptExpression.class}, new Object[] {null}); expressionProxies.add(abstractExpressionProxy); } } }; - - - public static ScriptProxyFactory getInstance(){ - if(isFinishScan.compareAndSet(false,true)){ + public static ScriptProxyFactory getInstance() { + if (isFinishScan.compareAndSet(false, true)) { expressionProxyFactory.scan.scanPackages("org.apache.rocketmq.streams.script.optimization.performance"); expressionProxyFactory.scan.scanPackages("org.apache.rocketmq.streams.filter.optimization"); } return expressionProxyFactory; } - public AbstractScriptProxy create(IScriptExpression oriScriptExpression){ - for(AbstractScriptProxy abstractExpressionProxy: expressionProxies){ + public AbstractScriptProxy create(IScriptExpression oriScriptExpression) { + for (AbstractScriptProxy abstractExpressionProxy : expressionProxies) { abstractExpressionProxy.setOrigExpression(oriScriptExpression); - if(abstractExpressionProxy.supportOptimization(oriScriptExpression)){ - return (AbstractScriptProxy)ReflectUtil.forInstance(abstractExpressionProxy.getClass(),new Class[]{IScriptExpression.class},new Object[]{oriScriptExpression}); + if (abstractExpressionProxy.supportOptimization(oriScriptExpression)) { + return (AbstractScriptProxy) ReflectUtil.forInstance(abstractExpressionProxy.getClass(), new Class[] {IScriptExpression.class}, new Object[] {oriScriptExpression}); } } return null; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/SimpleScriptExpressionProxy.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/SimpleScriptExpressionProxy.java index 249b10a5..84bf529a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/SimpleScriptExpressionProxy.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/optimization/performance/SimpleScriptExpressionProxy.java @@ -31,21 +31,23 @@ public abstract class SimpleScriptExpressionProxy extends AbstractScriptProxy { public SimpleScriptExpressionProxy(IScriptExpression origExpression) { super(origExpression); } - protected List optimizationExpressions=null; + + protected List optimizationExpressions = null; + @Override public List getCacheFilters() { - IScriptExpression scriptExpression=this.origExpression; - if(this.optimizationExpressions==null){ - synchronized (this){ - if(this.optimizationExpressions==null){ - List optimizationExpressions=new ArrayList<>(); - optimizationExpressions.add(new AbstractCacheFilter(getVarName(),this.origExpression) { + IScriptExpression scriptExpression = this.origExpression; + if (this.optimizationExpressions == null) { + synchronized (this) { + if (this.optimizationExpressions == null) { + List optimizationExpressions = new ArrayList<>(); + optimizationExpressions.add(new AbstractCacheFilter(getVarName(), this.origExpression) { @Override public boolean executeOrigExpression(IMessage message, AbstractContext context) { FunctionContext functionContext = new FunctionContext(message); if (context != null) { context.syncSubContext(functionContext); } - Boolean isMatch=(Boolean)scriptExpression.executeExpression(message,functionContext); + Boolean isMatch = (Boolean) scriptExpression.executeExpression(message, functionContext); if (context != null) { context.syncContext(functionContext); @@ -53,7 +55,7 @@ public List getCacheFilters() { return isMatch; } }); - this.optimizationExpressions=optimizationExpressions; + this.optimizationExpressions = optimizationExpressions; } } } @@ -61,15 +63,13 @@ public List getCacheFilters() { } - @Override public Object executeExpression(IMessage message, FunctionContext context) { - Boolean value= this.optimizationExpressions.get(0).execute(message,context); - if(this.origExpression.getNewFieldNames()!=null&&this.origExpression.getNewFieldNames().size()>0){ + Boolean value = this.optimizationExpressions.get(0).execute(message, context); + if (this.origExpression.getNewFieldNames() != null && this.origExpression.getNewFieldNames().size() > 0) { message.getMessageBody().put(this.origExpression.getNewFieldNames().iterator().next(), value); } return value; } - protected abstract String getVarName(); } 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 6b05827f..a75d300c 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 @@ -26,6 +26,7 @@ import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; + import com.alibaba.fastjson.JSONObject; import org.apache.commons.lang3.StringUtils; @@ -87,7 +88,7 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo /** * 用消息中的哪个字段做时间字段 */ - protected String timeFieldName ; + protected String timeFieldName; /** * having column in having clause eg: key:'having_sum_0001' value:'having_sum_0001=SUM(OrderPrice)<2000' note: here ignore the logical relation value may be multi expression which split by ${SCRIPT_SPLIT_CHAR} update: change sql(move the function into select clause) to escape function in having clause @@ -102,7 +103,7 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo /** * SQL中group by的字段,使用;拼接,如"name;age" */ - protected String groupByFieldName ; + protected String groupByFieldName; /** * 意义同blink中,允许最晚的消息到达时间,单位是分钟 @@ -120,7 +121,7 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo /** * 主要是做兼容,以前设计的窗口时间是分钟为单位,如果有秒作为窗口时间的,通过设置timeUntiAdjust=1来实现。 后续需要调整成直接秒级窗口 */ - protected int timeUnitAdjust=60; + protected int timeUnitAdjust = 60; /** * the variable name of window size which can be got from message */ @@ -144,14 +145,14 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo /** * 默认为空,窗口的触发类似flink,在测试模式下,因为消息有界,期望当消息发送完成后能触发,可以设置两条消息的最大间隔,超过这个间隔,将直接触发消息 */ - protected Long msgMaxGapSecond=10L; + protected Long msgMaxGapSecond = 10L; /** * 是否支持过期数据的计算 过期:当前时间大于数据所在窗口的触发时间 */ - protected int fireMode=0;//0:普通触发,firetime后收到数据丢弃;1:多实例多次独立触发,在watermark时间内,同starttime,endtime创建多个实例,多次触发;2.单实例,多次独立触发,每次触发是最新值 + protected int fireMode = 0;//0:普通触发,firetime后收到数据丢弃;1:多实例多次独立触发,在watermark时间内,同starttime,endtime创建多个实例,多次触发;2.单实例,多次独立触发,每次触发是最新值 - protected boolean isLocalStorageOnly=true;//是否只用本地存储,可以提高性能,但不保证可靠性 + protected boolean isLocalStorageOnly = true;//是否只用本地存储,可以提高性能,但不保证可靠性 protected String reduceSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码 protected transient IReducer reducer; /** @@ -205,12 +206,12 @@ protected boolean initConfigurable() { /** * 如果没有db配置,不开启远程存储服务 */ - if(!ORMUtil.hasConfigueDB()){ - isLocalStorageOnly=true; + if (!ORMUtil.hasConfigueDB()) { + isLocalStorageOnly = true; } - sqlCache=new SQLCache(isLocalStorageOnly); - AbstractWindow window=this; - windowCache=new WindowCache(){ + sqlCache = new SQLCache(isLocalStorageOnly); + AbstractWindow window = this; + windowCache = new WindowCache() { @Override protected String generateShuffleKey(IMessage message) { @@ -226,12 +227,12 @@ protected String generateShuffleKey(IMessage message) { initFunctionExecutor(); //启动shuffle channel 实现消息shuffle以及接收shuffle消息并处理 // FireManager.getInstance().startFireCheck(); - if(StringUtil.isNotEmpty(this.reduceSerializeValue)){ - byte[] bytes= Base64Utils.decode( this.reduceSerializeValue); + if (StringUtil.isNotEmpty(this.reduceSerializeValue)) { + byte[] bytes = Base64Utils.decode(this.reduceSerializeValue); reducer = InstantiationUtil.deserializeObject(bytes); } - eventTimeManager=new EventTimeManager(); - windowMaxValueManager = new WindowMaxValueManager(this,sqlCache); + eventTimeManager = new EventTimeManager(); + windowMaxValueManager = new WindowMaxValueManager(this, sqlCache); return success; } @@ -282,22 +283,22 @@ public String createWindowInstance(String startTime,String endTime,String fireTi } */ - public WindowInstance createWindowInstance(String startTime, String endTime, String fireTime,String splitId) { - WindowInstance windowInstance =new WindowInstance(); + public WindowInstance createWindowInstance(String startTime, String endTime, String fireTime, String splitId) { + WindowInstance windowInstance = new WindowInstance(); windowInstance.setFireTime(fireTime); windowInstance.setStartTime(startTime); windowInstance.setEndTime(endTime); windowInstance.setSplitId(splitId); windowInstance.setGmtCreate(new Date()); windowInstance.setGmtModified(new Date()); - windowInstance.setWindowInstanceName(createWindowInstanceName(startTime,endTime,fireTime)); + windowInstance.setWindowInstanceName(createWindowInstanceName(startTime, endTime, fireTime)); windowInstance.setWindowName(getConfigureName()); windowInstance.setWindowNameSpace(getNameSpace()); - String windowInstanceId =windowInstance.createWindowInstanceId(); + String windowInstanceId = windowInstance.createWindowInstanceId(); String dbWindowInstanceId = StringUtil.createMD5Str(windowInstanceId); windowInstance.setWindowInstanceKey(dbWindowInstanceId); - windowInstance.setWindowInstanceSplitName(StringUtil.createMD5Str(MapKeyUtil.createKey(getNameSpace(), getConfigureName(),splitId))); + windowInstance.setWindowInstanceSplitName(StringUtil.createMD5Str(MapKeyUtil.createKey(getNameSpace(), getConfigureName(), splitId))); windowInstance.setNewWindowInstance(true); return windowInstance; } @@ -310,8 +311,8 @@ public WindowInstance createWindowInstance(String startTime, String endTime, St * @param fireTime * @return */ - public String createWindowInstanceName(String startTime, String endTime, String fireTime){ - return fireMode==0?getConfigureName():fireTime; + public String createWindowInstanceName(String startTime, String endTime, String fireTime) { + return fireMode == 0 ? getConfigureName() : fireTime; } /** @@ -322,14 +323,14 @@ public String createWindowInstanceName(String startTime, String endTime, String * @return */ - public long incrementAndGetSplitNumber(WindowInstance instance,String shuffleId){ - long maxValue= windowMaxValueManager.incrementAndGetSplitNumber(instance,shuffleId); + public long incrementAndGetSplitNumber(WindowInstance instance, String shuffleId) { + long maxValue = windowMaxValueManager.incrementAndGetSplitNumber(instance, shuffleId); return maxValue; } public abstract Class getWindowBaseValueClass(); - public abstract int fireWindowInstance(WindowInstance windowInstance,MapqueueId2Offset) ; + public abstract int fireWindowInstance(WindowInstance windowInstance, Map queueId2Offset); /** * 计算每条记录的group by值,对于groupby分组,里面任何字段不能为null值,如果为null值,这条记录会被忽略 @@ -337,21 +338,21 @@ public long incrementAndGetSplitNumber(WindowInstance instance,String shuffleId) * @param message * @return */ - protected String generateShuffleKey(IMessage message){ + protected String generateShuffleKey(IMessage message) { if (StringUtil.isEmpty(groupByFieldName)) { return null; } - JSONObject msg=message.getMessageBody(); + JSONObject msg = message.getMessageBody(); String[] fieldNames = groupByFieldName.split(";"); - String[] values=new String[fieldNames.length]; + String[] values = new String[fieldNames.length]; boolean isFirst = true; - int i=0; + int i = 0; for (String filedName : fieldNames) { if (isFirst) { isFirst = false; } String value = msg.getString(filedName); - values[i]=value; + values[i] = value; i++; } return MapKeyUtil.createKey(values); @@ -359,8 +360,8 @@ protected String generateShuffleKey(IMessage message){ public abstract void clearFireWindowInstance(WindowInstance windowInstance); - public void clearFire(WindowInstance windowInstance){ - if(windowInstance==null){ + public void clearFire(WindowInstance windowInstance) { + if (windowInstance == null) { return; } clearFireWindowInstance(windowInstance); @@ -412,7 +413,7 @@ protected void initFunctionExecutor() { scriptBuilder = new StringBuilder(); } String[] functionParameterNames = scriptParameterList.stream().map( - scriptParameter -> scriptParameter.getScriptParameterStr()).collect(Collectors.toList()) + scriptParameter -> scriptParameter.getScriptParameterStr()).collect(Collectors.toList()) .toArray(new String[0]); AggregationScript accEngine = new AggregationScript( ((ScriptExpression)expression).getNewFieldName(), functionName, @@ -460,22 +461,23 @@ protected void initFunctionExecutor() { * @param message * @return */ - public List queryOrCreateWindowInstance(IMessage message,String queueId) { - return WindowInstance.getOrCreateWindowInstance(this, WindowInstance.getOccurTime(this, message), timeUnitAdjust, + public List queryOrCreateWindowInstance(IMessage message, String queueId) { + return WindowInstance.getOrCreateWindowInstance(this, WindowInstance.getOccurTime(this, message), timeUnitAdjust, queueId); } /** * 获取window处理的消息中最大的时间 + * * @param msg * @return */ - public void updateMaxEventTime(IMessage msg){ - eventTimeManager.updateEventTime(msg,this); + public void updateMaxEventTime(IMessage msg) { + eventTimeManager.updateEventTime(msg, this); } public Long getMaxEventTime(String queueId) { - return this.eventTimeManager.getMaxEventTime(queueId); + return this.eventTimeManager.getMaxEventTime(queueId); } /** @@ -483,15 +485,15 @@ public Long getMaxEventTime(String queueId) { * * @param windowValueList */ - public void sendFireMessage(List windowValueList,String queueId) { + public void sendFireMessage(List windowValueList, String queueId) { int count = 0; - List msgs=new ArrayList<>(); + List msgs = new ArrayList<>(); for (WindowValue windowValue : windowValueList) { JSONObject message = new JSONObject(); - if(JSONObject.class.isInstance(windowValue.getcomputedResult())){ - message=(JSONObject)windowValue.getcomputedResult(); - }else { + if (JSONObject.class.isInstance(windowValue.getcomputedResult())) { + message = (JSONObject)windowValue.getcomputedResult(); + } else { Iterator> it = windowValue.iteratorComputedColumnResult(); while (it.hasNext()) { Entry entry = it.next(); @@ -499,23 +501,23 @@ public void sendFireMessage(List windowValueList,String queueId) { } } - Long fireTime=DateUtil.parseTime(windowValue.getFireTime()).getTime(); - long baseTime= 1577808000000L ;//set base time from 2021-01-01 00:00:00 - int sameFireCount=0; - if(fireMode!=0){ - Long endTime=DateUtil.parseTime(windowValue.getEndTime()).getTime(); - sameFireCount=(int)((fireTime-endTime)/1000)/sizeInterval*timeUnitAdjust; - if(sameFireCount>=1){ - sameFireCount=1; + Long fireTime = DateUtil.parseTime(windowValue.getFireTime()).getTime(); + long baseTime = 1577808000000L;//set base time from 2021-01-01 00:00:00 + int sameFireCount = 0; + if (fireMode != 0) { + Long endTime = DateUtil.parseTime(windowValue.getEndTime()).getTime(); + sameFireCount = (int)((fireTime - endTime) / 1000) / sizeInterval * timeUnitAdjust; + if (sameFireCount >= 1) { + sameFireCount = 1; } } //can keep offset in order - Long offset=((fireTime-baseTime)/1000*10+sameFireCount)*100000000+windowValue.getPartitionNum(); - message.put("windowInstanceId",windowValue.getWindowInstancePartitionId()); - message.put("start_time",windowValue.getStartTime()); - message.put("end_time",windowValue.getEndTime()); - message.put("offset",offset); - Message newMessage=windowFireSource.createMessage(message,queueId,offset+"",false); + Long offset = ((fireTime - baseTime) / 1000 * 10 + sameFireCount) * 100000000 + windowValue.getPartitionNum(); + message.put("windowInstanceId", windowValue.getWindowInstancePartitionId()); + message.put("start_time", windowValue.getStartTime()); + message.put("end_time", windowValue.getEndTime()); + message.put("offset", offset); + Message newMessage = windowFireSource.createMessage(message, queueId, offset + "", false); newMessage.getHeader().setOffsetIsLong(true); if (count == windowValueList.size() - 1) { newMessage.getHeader().setNeedFlush(true); @@ -526,8 +528,8 @@ public void sendFireMessage(List windowValueList,String queueId) { count++; } - if(DebugWriter.getDebugWriter(this.getConfigureName()).isOpenDebug()){ - DebugWriter.getDebugWriter(this.getConfigureName()).writeWindowFire(this,msgs,queueId); + if (DebugWriter.getDebugWriter(this.getConfigureName()).isOpenDebug()) { + DebugWriter.getDebugWriter(this.getConfigureName()).writeWindowFire(this, msgs, queueId); } } @@ -687,7 +689,7 @@ public IReducer getReducer() { public void setReducer(IReducer reducer) { this.reducer = reducer; byte[] bytes = InstantiationUtil.serializeObject(reducer); - this.reduceSerializeValue=Base64Utils.encode(bytes); + this.reduceSerializeValue = Base64Utils.encode(bytes); } public int getTimeUnitAdjust() { @@ -752,8 +754,8 @@ public SQLCache getSqlCache() { return sqlCache; } - public void initWindowInstanceMaxSplitNum(WindowInstance instance){ - getWindowMaxValueManager().initMaxSplitNum(instance,queryWindowInstanceMaxSplitNum(instance)); + public void initWindowInstanceMaxSplitNum(WindowInstance instance) { + getWindowMaxValueManager().initMaxSplitNum(instance, queryWindowInstanceMaxSplitNum(instance)); } protected abstract Long queryWindowInstanceMaxSplitNum(WindowInstance instance); diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java index 03e4224e..4bcfe3a0 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; + import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack; import org.apache.rocketmq.streams.common.channel.sinkcache.impl.AbstractMultiSplitMessageCache; import org.apache.rocketmq.streams.db.driver.DriverBuilder; @@ -34,9 +35,9 @@ */ public class SQLCache extends AbstractMultiSplitMessageCache { - protected Boolean isOpenCache=true;//if false,then execute sql when receive sql - protected Set firedWindowInstances=new HashSet<>();//fired window instance ,if the owned sqls have not commit, can cancel the sqls - protected Map windowInstance2Index=new HashMap<>();//set index to ISQLElement group by window instance + protected Boolean isOpenCache = true;//if false,then execute sql when receive sql + protected Set firedWindowInstances = new HashSet<>();//fired window instance ,if the owned sqls have not commit, can cancel the sqls + protected Map windowInstance2Index = new HashMap<>();//set index to ISQLElement group by window instance protected boolean isLocalOnly; @@ -45,7 +46,7 @@ public SQLCache(boolean isLocalOnly) { this.isLocalOnly = isLocalOnly; this.flushCallBack = new MessageFlushCallBack(new SQLCacheCallback()); this.setBatchSize(1000); - this.setAutoFlushTimeGap(30 * 1000); + this.setAutoFlushTimeGap(10 * 1000); this.setAutoFlushSize(100); this.openAutoFlush(); } From 0742cf44bd5cf3903461f85fa0c9c4eef2d3c447 Mon Sep 17 00:00:00 2001 From: YUDA Date: Wed, 15 Sep 2021 11:37:25 +0800 Subject: [PATCH 3/5] Joinwindow bug fix (#61) * fix bugs * fix joinwindow message remove bugs --- pom.xml | 2 +- .../streams/common/topology/ChainStage.java | 4 +- .../common/topology/model/AbstractStage.java | 7 +- .../impl/string/SubStringIndexFunction.java | 6 + .../streams/window/model/WindowInstance.java | 4 + .../window/operator/impl/OverWindow.java | 2 +- .../window/operator/join/JoinWindow.java | 157 ++++++++++++------ .../window/shuffle/ShuffleChannel.java | 3 +- .../window/state/impl/WindowValue.java | 2 +- 9 files changed, 129 insertions(+), 58 deletions(-) diff --git a/pom.xml b/pom.xml index ab8825d2..7bbe1f42 100644 --- a/pom.xml +++ b/pom.xml @@ -70,7 +70,7 @@ 3.2.13.RELEASE 1.0-rc5 5.1.40 - 1.2.27 + 1.2.78 2.2.1 4.5.2 2.5 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 1916291d..d1b7ab6b 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 @@ -152,7 +152,9 @@ public void sendSystem(IMessage message, AbstractContext context, Pipeline... pi } Set set = new HashSet<>(); for (Pipeline pipeline : pipelines) { - set.add((ChainPipeline)pipeline); + if (pipeline != null) { + set.add((ChainPipeline)pipeline); + } } sendSystem(message, context, set); } 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 9860c51e..1a2d8b1a 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 @@ -84,7 +84,12 @@ public T doMessage(T t, AbstractContext context) { context.breakExecute(); return null; } - TraceUtil.debug(t.getHeader().getTraceId(), "AbstractStage", label, t.getMessageBody().toJSONString()); + 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); if (handle == null) { return t; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SubStringIndexFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SubStringIndexFunction.java index 705a5fc3..3d50f018 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SubStringIndexFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/string/SubStringIndexFunction.java @@ -85,6 +85,12 @@ public String substringindex(IMessage message, FunctionContext context, @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") Integer startIndex, @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") Integer endIndex) { oriMsg = FunctionUtils.getValueString(message, context, oriMsg); + int msgLength = oriMsg.length(); + if (startIndex >= msgLength) { + return ""; + } else if (endIndex > msgLength) { + endIndex = msgLength; + } return oriMsg.substring(startIndex, endIndex); } 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 ebcc0fe4..c5759767 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 @@ -109,6 +109,10 @@ public String createWindowInstanceId() { return MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime); } + public String createWindowInstanceIdWithoutSplitid() { + return MapKeyUtil.createKey(windowNameSpace, windowName, windowInstanceName, startTime, endTime); + } + public String createWindowInstanceTriggerId(){ return MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime,fireTime); } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java index b8b74f80..86fa99bf 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java @@ -128,7 +128,7 @@ public boolean isSynchronous() { @Override protected boolean initConfigurable() { - return true; + return super.initConfigurable(); } @Override 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 65796477..920d525d 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 @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.rocketmq.streams.common.utils.TraceUtil; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.Context; import org.apache.rocketmq.streams.common.context.IMessage; @@ -40,6 +41,7 @@ import org.apache.rocketmq.streams.window.state.impl.JoinLeftState; import org.apache.rocketmq.streams.window.state.impl.JoinRightState; import org.apache.rocketmq.streams.window.state.impl.JoinState; +import org.apache.rocketmq.streams.window.storage.ShufflePartitionManager; public class JoinWindow extends AbstractShuffleWindow { @@ -73,6 +75,7 @@ protected boolean initConfigurable() { // // } + @Override protected int fireWindowInstance(WindowInstance instance, String shuffleId, Map queueId2Offsets) { clearFire(instance); @@ -81,7 +84,8 @@ protected int fireWindowInstance(WindowInstance instance, String shuffleId, Map< @Override public void clearCache(String queueId) { - + getStorage().clearCache(shuffleChannel.getChannelQueue(queueId),getWindowBaseValueClass()); + ShufflePartitionManager.getInstance().clearSplit(queueId); } @Override @@ -110,8 +114,7 @@ public void shuffleCalculate(List messages, WindowInstance instance, S } for (IMessage msg : messages) { - MessageHeader header = JSONObject.parseObject(msg.getMessageBody(). - getString(WindowCache.ORIGIN_MESSAGE_HEADER), MessageHeader.class); + MessageHeader header = msg.getHeader(); String routeLabel = header.getMsgRouteFromLable(); // Map joinMessages = new HashMap<>(); String storeKeyPrefix = ""; @@ -128,7 +131,11 @@ public void shuffleCalculate(List messages, WindowInstance instance, S List tmpMessages = new ArrayList<>(); int count = 0; while (iterator.hasNext()) { - tmpMessages.add(iterator.next()); + WindowBaseValue windowBaseValue = iterator.next(); + if (windowBaseValue == null) { + continue; + } + tmpMessages.add(windowBaseValue); count++; if (count == 100) { sendMessage(msg, tmpMessages); @@ -146,7 +153,9 @@ private Iterator getMessageIterator(String queueId, WindowInsta List instances = new ArrayList<>(); for (Map.Entry entry : this.windowInstanceMap.entrySet()) { - instances.add(entry.getValue()); + if (queueId.equalsIgnoreCase(entry.getValue().getSplitId())) { + instances.add(entry.getValue()); + } } Iterator windowInstanceIter = instances.iterator(); return new Iterator() { @@ -159,9 +168,9 @@ public boolean hasNext() { if (iterator != null && iterator.hasNext()) { return true; } - while (windowInstanceIter.hasNext()) { + if (windowInstanceIter.hasNext()) { WindowInstance instance = windowInstanceIter.next(); - iterator = storage.loadWindowInstanceSplitData(null, queueId, + iterator = storage.loadWindowInstanceSplitData(null, null, instance.createWindowInstanceId(), keyPrefix, clazz); @@ -217,17 +226,64 @@ public WindowBaseValue next() { } - public List connectJoin(IMessage message, List> rows, String joinType, String rightAsName) { + public List connectJoin(IMessage message, List> rows, String joinType, + String rightAsName) { List result = new ArrayList<>(); if (rows.size() <= 0) { return result; } if ("inner".equalsIgnoreCase(joinType)) { result = connectInnerJoin(message, rows, rightAsName); + } else if ("left".equalsIgnoreCase(joinType)) { + result = connectLeftJoin(message, rows, rightAsName); + } + return result; + } + + private List connectLeftJoin(IMessage message, List> rows, String rightAsName) { + + List result = new ArrayList<>(); + String routeLabel = message.getHeader().getMsgRouteFromLable(); + JSONObject messageBody = message.getMessageBody(); + String traceId = message.getHeader().getTraceId(); + int index = 1; + if (LABEL_LEFT.equalsIgnoreCase(routeLabel) && rows.size() > 0) { + for (Map raw : rows) { + // addAsName(raw, rightAsName); + JSONObject object = (JSONObject)messageBody.clone(); + object.fluentPutAll(addAsName(raw, rightAsName)); + object.put(TraceUtil.TRACE_ID_FLAG, traceId + "-" + index); + index++; + result.add(object); + } + } else if (LABEL_LEFT.equalsIgnoreCase(routeLabel) && rows.size() <= 0) { + JSONObject object = (JSONObject) messageBody.clone(); + object.put(TraceUtil.TRACE_ID_FLAG, traceId + "-" + index); + result.add(object); + } else { + messageBody = addAsName(messageBody, rightAsName); + for (Map raw : rows) { + JSONObject object = (JSONObject)messageBody.clone(); + object.fluentPutAll(raw); + object.put(TraceUtil.TRACE_ID_FLAG, traceId + "-" + index); + index++; + result.add(object); + } + } + + + + if (rows != null && rows.size() > 0) { + for (Map raw : rows) { + JSONObject object = (JSONObject) messageBody.clone(); + object.fluentPutAll(raw); + result.add(object); + } + return result; + } + if (LABEL_LEFT.equalsIgnoreCase(routeLabel)) { + result.add(messageBody); } - // else if ("left".equalsIgnoreCase(joinType)) { - // result = connectLeftJoin(message, rows, rightAsName); - // } return result; } @@ -241,12 +297,16 @@ public List connectJoin(IMessage message, List> public List connectInnerJoin(IMessage message, List> rows, String rightAsName) { List result = new ArrayList<>(); String routeLabel = message.getHeader().getMsgRouteFromLable(); + String traceId = message.getHeader().getTraceId(); + int index = 1; if (LABEL_LEFT.equalsIgnoreCase(routeLabel)) { JSONObject messageBody = message.getMessageBody(); for (Map raw : rows) { // addAsName(raw, rightAsName); JSONObject object = (JSONObject)messageBody.clone(); object.fluentPutAll(addAsName(raw, rightAsName)); + object.put(TraceUtil.TRACE_ID_FLAG, traceId + "-" + index); + index++; result.add(object); } } else { @@ -255,6 +315,8 @@ public List connectInnerJoin(IMessage message, List raw : rows) { JSONObject object = (JSONObject)messageBody.clone(); object.fluentPutAll(raw); + object.put(TraceUtil.TRACE_ID_FLAG, traceId + "-" + index); + index++; result.add(object); } } @@ -284,14 +346,9 @@ private JSONObject addAsName(Map raw, String rightAsName) { */ protected String createStoreKey(IMessage message, String routeLabel, WindowInstance windowInstance) { String shuffleKey = message.getMessageBody().getString(WindowCache.SHUFFLE_KEY); - String shuffleId = shuffleChannel.getChannelQueue(shuffleKey).getQueueId(); String orginQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID); String originOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET); - String windowNamespace = getNameSpace(); - String windowName = getConfigureName(); - String startTime = windowInstance.getStartTime(); - String endTime = windowInstance.getEndTime(); - String storeKey = MapKeyUtil.createKey(shuffleId, windowNamespace, windowName, startTime, endTime, shuffleKey, routeLabel, orginQueueId, originOffset); + String storeKey = MapKeyUtil.createKey(windowInstance.createWindowInstanceId(), shuffleKey, routeLabel, orginQueueId, originOffset); return storeKey; } @@ -327,6 +384,8 @@ private JoinState createJoinState(IMessage message, WindowInstance instance, Str JSONObject messageBody = (JSONObject)message.getMessageBody().clone(); messageBody.remove("WindowInstance"); messageBody.remove("AbstractWindow"); + messageBody.remove(WindowCache.ORIGIN_MESSAGE_HEADER); + messageBody.remove("MessageHeader"); JoinState state = null; if ("left".equalsIgnoreCase(routeLabel)) { @@ -396,47 +455,41 @@ public Class getWindowBaseValueClass() { return JoinState.class; } - // @Override - // public void finishWindowProcessAndSend2Receiver(List messageList,WindowInstance windowInstance) { - // for (IMessage message : messageList) { - // List> result = joinOperator.dealJoin(message); - // List> rows = matchRows(message.getMessageBody(), result); - // String rightAsName = message.getMessageBody().getString("rightAsName"); - // String joinType = message.getMessageBody().getString("joinType"); - // List connectMsgs = joinOperator.connectJoin(message, rows, joinType, rightAsName); - // for (int i=0; i < connectMsgs.size(); i++) { - // if (i == connectMsgs.size() -1) { - // sendMessage(connectMsgs.get(i), true); - // } else { - // sendMessage(connectMsgs.get(i), false); - // } - // - // } - // - // } - // //todo 完成处理 - // //todo 发送消息到下一个节点 sendFireMessage(); - // } /** * window触发后的清理工作 - * @param windowInstances - */ - /** - * 删除掉触发过的数据 - * - * @param instance + * @param windowInstance */ @Override - public void clearFireWindowInstance(WindowInstance instance) { - if(instance==null){ - return; + public void clearFireWindowInstance(WindowInstance windowInstance) { +// String partitionNum=(getOrderBypPrefix()+ windowInstance.getSplitId()); + + List removeInstances = new ArrayList<>(); + + Date clearTime = DateUtil.addSecond(DateUtil.parse(windowInstance.getStartTime()), -sizeInterval * (retainWindowCount - 1) * 60); + Iterator iterable = this.windowInstanceMap.keySet().iterator(); + while (iterable.hasNext()) { + WindowInstance instance = this.windowInstanceMap.get(iterable.next()); + Date startTime = DateUtil.parse(instance.getStartTime()); + if (DateUtil.dateDiff(clearTime, startTime) >= 0) { + removeInstances.add(instance); + iterable.remove(); + } } - WindowInstance.clearInstance(instance); - joinOperator.cleanMessage(instance.getWindowNameSpace(), instance.getWindowName(), this.getRetainWindowCount(), - this.getSizeInterval(), instance.getStartTime()); - //todo windowinstace - //todo left+right + + for (WindowInstance instance : removeInstances) { + + windowMaxValueManager.deleteSplitNum(instance, instance.getSplitId()); + ShufflePartitionManager.getInstance().clearWindowInstance(instance.createWindowInstanceId()); + storage.delete(instance.createWindowInstanceId(),null,WindowBaseValue.class,sqlCache); + if(!isLocalStorageOnly){ + WindowInstance.clearInstance(instance,sqlCache); + joinOperator.cleanMessage(instance.getWindowNameSpace(), instance.getWindowName(), this.getRetainWindowCount(), + this.getSizeInterval(), windowInstance.getStartTime()); + } + } + + } protected List> matchRows(JSONObject msg, List> rows) { 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 cb6cb4e3..ba17a442 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 @@ -67,7 +67,7 @@ public class ShuffleChannel extends AbstractSystemChannel { protected static final Log LOG = LogFactory.getLog(ShuffleChannel.class); protected static final String SHUFFLE_QUEUE_ID = "SHUFFLE_QUEUE_ID"; - + protected static final String SHUFFLE_OFFSET = "SHUFFLE_OFFSET"; protected static final String SHUFFLE_MESSAGES = "SHUFFLE_MESSAGES"; protected String MSG_OWNER = "MSG_OWNER";//消息所属的window @@ -158,6 +158,7 @@ public Object doMessage(IMessage oriMessage, AbstractContext context) { for (Object obj : messages) { IMessage message = new Message((JSONObject) obj); message.getHeader().setQueueId(queueId); + message.getMessageBody().put(SHUFFLE_OFFSET, oriMessage.getHeader().getOffset()); window.updateMaxEventTime(message); if (isRepeateMessage(message, queueId)) { continue; 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 b4ae9042..e04bd161 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 @@ -264,7 +264,7 @@ public boolean calculate(AbstractWindow window, IMessage message) { calProjectColumn(window, message); String traceId = message.getMessageBody().getString(WindowCache.ORIGIN_MESSAGE_TRACE_ID); if (!StringUtil.isEmpty(traceId)) { - TraceUtil.debug(traceId, "window value result", getComputedColumnResult()); + TraceUtil.debug(traceId, "window value result", decodeSQLContent(getComputedColumnResult())); } } catch (Exception e) { LOG.error("failed in calculating the message", e); From 87a32224d2a874ba50dac4aee881237f0f62c8ae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=9B=B6=E5=8F=B7=E7=A8=8B=E5=BA=8F?= Date: Wed, 15 Sep 2021 11:45:39 +0800 Subject: [PATCH 4/5] Add Sqlmode (#63) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * add update logic for the DBSinker 、 upgrade the concat_ws function * Add the field level cache to reduce duplicate data entry #60 * Sqlmode can be used to write data to the database as specified #62 Co-authored-by: junjie.cheng --- .../rocketmq/streams/db/sink/DBSink.java | 184 ++++++++++-------- .../streams/db/sink/DBSinkBuilder.java | 24 +-- .../streams/client/transform/DataStream.java | 70 +++---- .../common/metadata/AbstractMetaData.java | 2 +- .../streams/common/metadata/MetaData.java | 2 +- .../common/metadata/MetaDataField.java | 32 ++- .../topology/stages/OutputChainStage.java | 5 +- 7 files changed, 167 insertions(+), 152 deletions(-) diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java index 1aab2bab..886e23ae 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSink.java @@ -18,40 +18,34 @@ import com.alibaba.fastjson.JSONObject; import com.google.common.collect.Lists; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.util.List; +import java.util.Locale; +import java.util.Set; import org.apache.rocketmq.streams.common.channel.IChannel; import org.apache.rocketmq.streams.common.channel.sink.AbstractSink; import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache; -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.component.AbstractComponent; import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.common.metadata.MetaData; +import org.apache.rocketmq.streams.common.metadata.MetaDataField; import org.apache.rocketmq.streams.common.utils.SQLUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.db.driver.DriverBuilder; import org.apache.rocketmq.streams.db.driver.JDBCDriver; -import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.List; -import java.util.Set; - /** * 主要用于写db,输入可以是一个insert/replace 模版,也可以是metadata对象,二者选一即可。都支持批量插入,提高吞吐 sql 模版:insert into table(column1,column2,column3)values('#{var1}',#{var2},'#{var3}') MetaData:主要是描述每个字段的类型,是否必须 二者选一个即可。sql模式,系统会把一批(batchSize)数据拼成一个大sql。metadata模式,基于字段描述,最终也是拼成一个大sql */ public class DBSink extends AbstractSink { - - protected String insertSQLTemplate;//完成插入部分的工作,和metadata二选一。insert into table(column1,column2,column3)values('#{var1}',#{var2},'#{var3}') - - protected String duplicateSQLTemplate; //通过on duplicate key update 来对已经存在的信息进行更新 - - protected MetaData metaData;//可以指定meta data,和insertSQL二选一 - - protected String tableName; //指定要插入的数据表 + public static final String SQL_MODE_DEFAULT = "default"; + public static final String SQL_MODE_REPLACE = "replace"; + public static final String SQL_MODE_IGNORE = "ignore"; @ENVDependence protected String jdbcDriver = AbstractComponent.DEFAULT_JDBC_DRIVER; @@ -60,7 +54,15 @@ public class DBSink extends AbstractSink { @ENVDependence protected String userName; @ENVDependence + protected String tableName; //指定要插入的数据表 + @ENVDependence protected String password; + @ENVDependence + protected String sqlMode; + + protected MetaData metaData;//可以指定meta data,和insertSQL二选一 + + protected String insertSQLTemplate;//完成插入部分的工作,和metadata二选一。insert into table(column1,column2,column3)values('#{var1}',#{var2},'#{var3}') protected boolean openSqlCache = true; @@ -87,60 +89,83 @@ public DBSink(String insertSQL, String dbInfoNamePrefix) { } public DBSink() { - setType(IChannel.TYPE); + this(null, null, null, null); } - public DBSink(String url, String userName, String password) { - setType(IChannel.TYPE); - this.url = url; - this.userName = userName; - this.password = password; + public DBSink(String url, String userName, String password, String tableName) { + this(url, userName, password, tableName, SQL_MODE_DEFAULT); } - public DBSink(String insertSQL, String url, String userName, String password) { + public DBSink(String url, String userName, String password, String tableName, String sqlMode) { + this(url, userName, password, tableName, sqlMode, null); + } + + public DBSink(String url, String userName, String password, String tableName, String sqlMode, MetaData metaData) { setType(IChannel.TYPE); this.url = url; this.userName = userName; this.password = password; - this.insertSQLTemplate = insertSQL; + this.tableName = tableName; + this.sqlMode = sqlMode; + this.metaData = metaData; } @Override protected boolean initConfigurable() { - try { - Class.forName("com.mysql.jdbc.Driver"); - if (StringUtil.isNotEmpty(this.tableName)) { - Connection connection = DriverManager.getConnection(url, userName, password); - DatabaseMetaData metaData = connection.getMetaData(); - ResultSet metaResult = metaData.getColumns(connection.getCatalog(), "%", this.tableName, null); - this.metaData = MetaData.createMetaData(metaResult); - this.metaData.setTableName(this.tableName); - } - sqlCache = new MessageCache<>(new IMessageFlushCallBack() { - @Override - public boolean flushMessage(List sqls) { - JDBCDriver dataSource = DriverBuilder.createDriver(jdbcDriver, url, userName, password); - try { - dataSource.executSqls(sqls); - } catch (Exception e) { - e.printStackTrace(); - throw new RuntimeException(e); - } finally { - if (dataSource != null) { - dataSource.destroy(); - } - } - return true; + if (this.metaData == null) { + try { + Class.forName("com.mysql.jdbc.Driver"); + if (StringUtil.isNotEmpty(this.tableName)) { + Connection connection = DriverManager.getConnection(this.url, this.userName, this.password); + DatabaseMetaData connectionMetaData = connection.getMetaData(); + ResultSet metaResult = connectionMetaData.getColumns(connection.getCatalog(), "%", this.tableName, null); + this.metaData = MetaData.createMetaData(metaResult); + this.metaData.setTableName(this.tableName); } - }); - ((MessageCache) sqlCache).setAutoFlushTimeGap(100000); - ((MessageCache) sqlCache).setAutoFlushSize(50); - sqlCache.openAutoFlush(); - return super.initConfigurable(); - } catch (ClassNotFoundException | SQLException e) { - e.printStackTrace(); + } catch (Exception e) { + e.printStackTrace(); + } + } + List fieldList = this.metaData.getMetaDataFields(); + List insertFields = Lists.newArrayList(); + List insertValues = Lists.newArrayList(); + List duplicateKeys = Lists.newArrayList(); + fieldList.forEach(field -> { + String fieldName = field.getFieldName(); + insertFields.add(fieldName); + insertValues.add("'#{" + fieldName + "}'"); + duplicateKeys.add(fieldName + " = VALUES(" + fieldName + ")"); + }); + + String sql = "insert"; + if (sqlMode == null || SQL_MODE_DEFAULT.equals(sqlMode)) { + sql = sql + " into "; + } else if (SQL_MODE_IGNORE.equals(sqlMode)) { + sql = sql + " ignore into "; + } else if (SQL_MODE_REPLACE.equals(sqlMode)) { + sql = sql + " into "; } - return false; + sql = sql + tableName + "(" + String.join(",", insertFields) + ") values (" + String.join(",", insertValues) + ") "; + if (SQL_MODE_REPLACE.equals(sqlMode)) { + sql = sql + " on duplicate key update " + String.join(",", duplicateKeys); + } + this.insertSQLTemplate = sql; + this.sqlCache = new MessageCache<>(sqls -> { + JDBCDriver dataSource = DriverBuilder.createDriver(jdbcDriver, url, userName, password); + try { + dataSource.executSqls(sqls); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + dataSource.destroy(); + } + return true; + }); + ((MessageCache) this.sqlCache).setAutoFlushTimeGap(100000); + ((MessageCache) this.sqlCache).setAutoFlushSize(50); + this.sqlCache.openAutoFlush(); + return super.initConfigurable(); } @Override @@ -154,7 +179,6 @@ protected boolean batchInsert(List messageList) { if (StringUtil.isEmpty(insertSQLTemplate) && metaData != null) { String sql = SQLUtil.createInsertSql(metaData, messages.get(0)); sql += SQLUtil.createInsertValuesSQL(metaData, messages.subList(1, messages.size())); - sql += this.duplicateSQLTemplate; executeSQL(dbDataSource, sql); return true; } @@ -162,7 +186,6 @@ protected boolean batchInsert(List messageList) { if (StringUtil.isEmpty(insertValueSQL) || insertSQLTemplate.replace(insertValueSQL, "").contains("#{")) { for (JSONObject message : messages) { String sql = parseSQL(message, insertSQLTemplate); - sql += this.duplicateSQLTemplate; executeSQL(dbDataSource, sql); } return true; @@ -172,7 +195,6 @@ protected boolean batchInsert(List messageList) { subInsert.add(parseSQL(message, insertValueSQL)); } String insertSQL = this.insertSQLTemplate.replace(insertValueSQL, String.join(",", subInsert)); - insertSQL += this.duplicateSQLTemplate; executeSQL(dbDataSource, insertSQL); return true; } @@ -195,7 +217,16 @@ protected void executeSQL(JDBCDriver dbDataSource, String sql) { } else { dbDataSource.execute(sql); } + } + protected void executeSQL(JDBCDriver dbDataSource, List sqls) { + if (isOpenSqlCache()) { + for (String sql : sqls) { + this.sqlCache.addCache(sql); + } + } else { + dbDataSource.executSqls(sqls); + } } /** @@ -209,7 +240,7 @@ protected String parseInsertValues(String insertSQL) { return null; } String valuesSQL = insertSQL.substring(start + VALUES_NAME.length()); - int end = valuesSQL.toLowerCase().lastIndexOf(")"); + int end = valuesSQL.toLowerCase().indexOf(")"); if (end == -1) { return null; } @@ -260,14 +291,6 @@ public void setPassword(String password) { this.password = password; } - public MetaData getMetaData() { - return metaData; - } - - public void setMetaData(MetaData metaData) { - this.metaData = metaData; - } - public String getTableName() { return tableName; } @@ -276,6 +299,22 @@ public void setTableName(String tableName) { this.tableName = tableName; } + public String getSqlMode() { + return sqlMode; + } + + public void setSqlMode(String sqlMode) { + this.sqlMode = sqlMode; + } + + public MetaData getMetaData() { + return metaData; + } + + public void setMetaData(MetaData metaData) { + this.metaData = metaData; + } + public boolean isOpenSqlCache() { return openSqlCache; } @@ -284,11 +323,4 @@ public void setOpenSqlCache(boolean openSqlCache) { this.openSqlCache = openSqlCache; } - public String getDuplicateSQLTemplate() { - return duplicateSQLTemplate; - } - - public void setDuplicateSQLTemplate(String duplicateSQLTemplate) { - this.duplicateSQLTemplate = duplicateSQLTemplate; - } } diff --git a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java index 87f5fb77..1743f627 100644 --- a/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java +++ b/rocketmq-streams-channel-db/src/main/java/org/apache/rocketmq/streams/db/sink/DBSinkBuilder.java @@ -17,17 +17,13 @@ package org.apache.rocketmq.streams.db.sink; import com.google.auto.service.AutoService; -import com.google.common.collect.Lists; +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.metadata.MetaDataField; import org.apache.rocketmq.streams.common.model.ServiceName; -import java.util.List; -import java.util.Properties; - @AutoService(IChannelBuilder.class) @ServiceName(DBSinkBuilder.TYPE) public class DBSinkBuilder implements IChannelBuilder { @@ -39,21 +35,9 @@ public ISink createSink(String namespace, String name, Properties properties, Me sink.setUrl(properties.getProperty("url")); sink.setUserName(properties.getProperty("userName")); sink.setPassword(properties.getProperty("password")); - List fieldList = metaData.getMetaDataFields(); - - List insertFields = Lists.newArrayList(); - List insertValues = Lists.newArrayList(); - List duplicateKeys = Lists.newArrayList(); - fieldList.forEach(field -> { - String fieldName = field.getFieldName(); - insertFields.add(fieldName); - insertValues.add("'#{" + fieldName + "}'"); - duplicateKeys.add(fieldName + " = VALUES(" + fieldName + ")"); - }); - - String sql = "insert into " + properties.getProperty("tableName") + "(" + String.join(",", insertFields) + ") values (" + String.join(",", insertValues) + ") "; - sink.setInsertSQLTemplate(sql); - sink.setDuplicateSQLTemplate(" on duplicate key update " + String.join(",", duplicateKeys)); + sink.setTableName(properties.getProperty("tableName")); + sink.setSqlMode(properties.getProperty("sqlMode")); + sink.setMetaData(metaData); return sink; } 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 f9260eab..08bfe7f0 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 @@ -73,7 +73,8 @@ public DataStream(PipelineBuilder pipelineBuilder, ChainStage currentChainSta this.currentChainStage = currentChainStage; } - public DataStream(PipelineBuilder pipelineBuilder, Set pipelineBuilders, ChainStage currentChainStage) { + public DataStream(PipelineBuilder pipelineBuilder, Set pipelineBuilders, + ChainStage currentChainStage) { this.mainPipelineBuilder = pipelineBuilder; this.otherPipelineBuilders = pipelineBuilders; this.currentChainStage = currentChainStage; @@ -97,11 +98,11 @@ public DataStream map(MapFunction mapFunction) { @Override protected T operate(IMessage message, AbstractContext context) { try { - O o = (O)(message.getMessageValue()); - T result = (T)mapFunction.map(o); + O o = (O) (message.getMessageValue()); + T result = (T) mapFunction.map(o); if (result != message.getMessageValue()) { if (result instanceof JSONObject) { - message.setMessageBody((JSONObject)result); + message.setMessageBody((JSONObject) result); } else { message.setMessageBody(new UserDefinedMessage(result)); } @@ -118,28 +119,28 @@ protected T operate(IMessage message, AbstractContext context) { return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage); } - public DataStream flatMap(FlatMapFunction mapFunction) { StageBuilder stageBuilder = new StageBuilder() { @Override protected T operate(IMessage message, AbstractContext context) { try { - O o = (O)(message.getMessageValue()); - List result =(List)mapFunction.flatMap(o); - if(result==null||result.size()==0){ + O o = (O) (message.getMessageValue()); + List result = (List) mapFunction.flatMap(o); + if (result == null || result.size() == 0) { context.breakExecute(); } - List splitMessages=new ArrayList<>(); - for(T t:result){ - Message subMessage=null; + List splitMessages = new ArrayList<>(); + for (T t : result) { + Message subMessage = null; if (result instanceof JSONObject) { - subMessage=new Message((JSONObject)t); + subMessage = new Message((JSONObject) t); } else { - subMessage=new Message(new UserDefinedMessage(t)); + subMessage = new Message(new UserDefinedMessage(t)); } splitMessages.add(subMessage); } - context.openSplitModel();; + context.openSplitModel(); + ; context.setSplitMessages(splitMessages); return null; } catch (Exception e) { @@ -159,7 +160,7 @@ public DataStream filter(final FilterFunction filterFunction) { @Override protected T operate(IMessage message, AbstractContext context) { try { - boolean isFilter = filterFunction.filter((O)message.getMessageValue()); + boolean isFilter = filterFunction.filter((O) message.getMessageValue()); if (isFilter) { context.breakExecute(); } @@ -232,7 +233,7 @@ public DataStream union(DataStream rightStream) { Union union = new Union(); //处理左流,做流的isMain设置成true - UDFUnionChainStage chainStage = (UDFUnionChainStage)this.mainPipelineBuilder.createStage(union); + UDFUnionChainStage chainStage = (UDFUnionChainStage) this.mainPipelineBuilder.createStage(union); chainStage.setMainStream(true); this.mainPipelineBuilder.setTopologyStages(currentChainStage, chainStage); @@ -272,7 +273,8 @@ protected T operate(IMessage message, AbstractContext context) { * @param sqlOrTableName * @return */ - public JoinStream join(String url, String userName, String password, String sqlOrTableName, long pollingTimeMintue) { + public JoinStream join(String url, String userName, String password, String sqlOrTableName, + long pollingTimeMintue) { return join(url, userName, password, sqlOrTableName, null, pollingTimeMintue); } @@ -285,7 +287,8 @@ public JoinStream join(String url, String userName, String password, String sqlO * @param sqlOrTableName * @return */ - public JoinStream join(String url, String userName, String password, String sqlOrTableName, String jdbcDriver, long pollingTimeMinute) { + public JoinStream join(String url, String userName, String password, String sqlOrTableName, String jdbcDriver, + long pollingTimeMinute) { DBDim dbDim = new DBDim(); dbDim.setUrl(url); dbDim.setUserName(userName); @@ -308,7 +311,7 @@ public DataStream forEach(ForEachFunction forEachFunction) { StageBuilder selfChainStage = new StageBuilder() { @Override protected T operate(IMessage message, AbstractContext context) { - forEachFunction.foreach((O)message.getMessageValue()); + forEachFunction.foreach((O) message.getMessageValue()); return null; } }; @@ -367,7 +370,6 @@ protected void start(boolean isAsyn) { return; } - ConfigurableComponent configurableComponent = ComponentCreator.getComponent(mainPipelineBuilder.getPipelineNameSpace(), ConfigurableComponent.class, ConfigureFileKey.CONNECT_TYPE + ":memory"); ChainPipeline pipeline = this.mainPipelineBuilder.build(configurableComponent.getService()); pipeline.startChannel(); @@ -403,21 +405,23 @@ protected void addOtherDataStream(DataStream rightSource) { this.otherPipelineBuilders.addAll(rightSource.otherPipelineBuilders); } - public DataStreamAction toFile(String filePath,int batchSize,boolean isAppend) { - FileSink fileChannel = new FileSink(filePath,isAppend); - if(batchSize>0){ + public DataStreamAction toFile(String filePath, int batchSize, boolean isAppend) { + FileSink fileChannel = new FileSink(filePath, isAppend); + if (batchSize > 0) { fileChannel.setBatchSize(batchSize); } ChainStage output = mainPipelineBuilder.createStage(fileChannel); mainPipelineBuilder.setTopologyStages(currentChainStage, output); return new DataStreamAction(this.mainPipelineBuilder, this.otherPipelineBuilders, output); } - public DataStreamAction toFile(String filePath,boolean isAppend) { - FileSink fileChannel = new FileSink(filePath,isAppend); + + public DataStreamAction toFile(String filePath, boolean isAppend) { + FileSink fileChannel = new FileSink(filePath, isAppend); ChainStage output = mainPipelineBuilder.createStage(fileChannel); mainPipelineBuilder.setTopologyStages(currentChainStage, output); return new DataStreamAction(this.mainPipelineBuilder, this.otherPipelineBuilders, output); } + public DataStreamAction toFile(String filePath) { FileSink fileChannel = new FileSink(filePath); ChainStage output = mainPipelineBuilder.createStage(fileChannel); @@ -440,28 +444,26 @@ public DataStreamAction toPrint(int batchSize) { } public DataStreamAction toDB(String url, String userName, String password, String tableName) { - DBSink dbChannel = new DBSink(url, userName, password); - dbChannel.setTableName(tableName); + DBSink dbChannel = new DBSink(url, userName, password, tableName); ChainStage output = this.mainPipelineBuilder.createStage(dbChannel); this.mainPipelineBuilder.setTopologyStages(currentChainStage, output); return new DataStreamAction(this.mainPipelineBuilder, this.otherPipelineBuilders, output); } public DataStreamAction toRocketmq(String topic) { - return toRocketmq(topic, "*", null,-1, null); + return toRocketmq(topic, "*", null, -1, null); } - - public DataStreamAction toRocketmq(String topic,String namesrvAddr) { - return toRocketmq(topic, "*", null,-1, namesrvAddr); + public DataStreamAction toRocketmq(String topic, String namesrvAddr) { + return toRocketmq(topic, "*", null, -1, namesrvAddr); } public DataStreamAction toRocketmq(String topic, String tags, - String namesrvAddr) { - return toRocketmq(topic, tags,null,-1, namesrvAddr); + String namesrvAddr) { + return toRocketmq(topic, tags, null, -1, namesrvAddr); } - public DataStreamAction toRocketmq(String topic, String tags,String groupName, int batchSize, String namesrvAddr) { + public DataStreamAction toRocketmq(String topic, String tags, String groupName, int batchSize, String namesrvAddr) { RocketMQSink rocketMQSink = new RocketMQSink(); rocketMQSink.setTopic(topic); rocketMQSink.setTags(tags); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/AbstractMetaData.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/AbstractMetaData.java index 7efd6d3a..59f7464e 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/AbstractMetaData.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/AbstractMetaData.java @@ -197,7 +197,7 @@ public String toJson() { Iterator i$ = this.metaDataFields.iterator(); while (i$.hasNext()) { - MetaDataField field = (MetaDataField)i$.next(); + MetaDataField field = (MetaDataField) i$.next(); jsonArray.add(field.toJson()); } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaData.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaData.java index 3deec903..b4f04d3c 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaData.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaData.java @@ -164,7 +164,7 @@ public static MetaData createMetaDate(Object object, Map paras) } public static String getTableName(Class clazz) { - TableClassName tableClass = (TableClassName)clazz.getAnnotation(TableClassName.class); + TableClassName tableClass = (TableClassName) clazz.getAnnotation(TableClassName.class); if (tableClass != null) { String className = tableClass.value(); if (StringUtil.isNotEmpty(className)) { diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataField.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataField.java index 9ec882ba..5a035ae9 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataField.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/metadata/MetaDataField.java @@ -116,7 +116,7 @@ public String toJson() { JSONObject jsonObject = new JSONObject(); jsonObject.put("fieldName", fieldName); if (dataType == null) { - dataType = (DataType)new StringDataType(); + dataType = (DataType) new StringDataType(); } jsonObject.put("dataType", dataType.toJson()); jsonObject.put("isRequired", isRequired); @@ -134,8 +134,8 @@ public void toObject(String jsonString) { this.isPrimary = jsonObject.getBoolean("isPrimary"); } - public static DataType getDataTypeByStr(String dataType) { - DataType dt = null; + public static DataType getDataTypeByStr(String dataType) { + DataType dt = null; if ("String".equals(dataType)) { dt = new StringDataType(); } else if ("long".equals(dataType)) { @@ -152,22 +152,20 @@ public static DataType getDataTypeByStr(String dataType) { return dt; } - public static String getDataTypeStrByType(DataType dataType) { - String dataTypestr = ""; - if (StringDataType.class.isInstance(dataType)) { - dataTypestr = "String"; - } else if (LongDataType.class.isInstance(dataType)) { - dataTypestr = "long"; - } else if (IntDataType.class.isInstance(dataType)) { - dataTypestr = "int"; - } else if (FloatDataType.class.isInstance(dataType)) { - dataTypestr = "float"; - } else if (Boolean.class.isInstance(dataType)) { - dataTypestr = "boolean"; + public static String getDataTypeStrByType(DataType dataType) { + String dataTypeStr = ""; + if (dataType instanceof StringDataType) { + dataTypeStr = "String"; + } else if (dataType instanceof LongDataType) { + dataTypeStr = "long"; + } else if (dataType instanceof IntDataType) { + dataTypeStr = "int"; + } else if (dataType instanceof FloatDataType) { + dataTypeStr = "float"; } else { - dataTypestr = "String"; + dataTypeStr = "String"; } - return dataTypestr; + return dataTypeStr; } } 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 5cbac0f8..a715ca3d 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 @@ -82,13 +82,12 @@ protected IMessage doProcess(IMessage message, AbstractContext context) { */ if (openMockChannel()) { if (mockSink != null) { - mockSink.batchAdd(message.deepCopy()); + mockSink.batchAdd(message); return message; } return message; } - sink.batchAdd(message.deepCopy()); - + sink.batchAdd(message); return message; } From bb22e674ff07c6db16fde57430b50cc2e4328f8f Mon Sep 17 00:00:00 2001 From: yuanxiaodongn Date: Thu, 16 Sep 2021 16:47:38 +0800 Subject: [PATCH 5/5] fixed bugs --- .../rocketmq/streams/client/SourceTest.java | 8 ++ .../sink/AbstractSupportShuffleSink.java | 6 +- .../common/channel/source/AbstractSource.java | 10 ++ .../common/optimization/HyperscanRegex.java | 64 ++++++--- .../common/optimization/LikeRegex.java | 68 +++++++++- .../optimization/LogFingerprintFilter.java | 3 +- .../optimization/OptimizationRegex.java | 6 +- .../streams/common/utils/ReflectUtil.java | 3 + .../filter/engine/impl/DefaultRuleEngine.java | 2 +- .../streams/filter/operator/Rule.java | 4 +- .../operator/expression/GroupExpression.java | 13 +- .../expression/GroupExpressionManager.java | 15 ++- .../PiplineLogFingerprintAnalysis.java | 5 +- .../streams/lease/LeaseComponentTest.java | 4 +- .../function/impl/json/JsonValueFunction.java | 3 + .../operator/expression/ScriptExpression.java | 42 +----- .../service/impl/ScriptServiceImpl.java | 3 + .../script/service/udf/UDAFScript.java | 4 - .../window/operator/AbstractWindow.java | 122 +++++++++--------- .../streams/window/sqlcache/SQLCache.java | 6 +- 20 files changed, 248 insertions(+), 143 deletions(-) diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java index 3be803b3..ccc8589c 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java @@ -48,4 +48,12 @@ public void testImportMsgFromSource(){ .toFile("/tmp/aegis_proc.txt",true) .start(); } + + @Test + public void testImportMsgFromNet(){ + DataStreamSource.create("tmp","tmp") + .fromRocketmq("TOPIC_AEGIS_DETECT_MSG","chris_test","T_MSG_NETSTAT",true,null) + .toFile("/tmp/aegis_net.txt",true) + .start(); + } } 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 b2a1c461..5a19bfbd 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 = 10;//分片个数 + protected transient int splitNum ;//分片个数 //sls对应的project和logstore初始化是否完成标志 protected volatile transient boolean hasCreated = false; @@ -49,7 +49,9 @@ public abstract class AbstractSupportShuffleSink extends AbstractSink { protected boolean initConfigurable() { boolean success = super.initConfigurable(); hasCreated = false; - checkAndCreateTopic(); + if(this.splitNum>0){ + checkAndCreateTopic(); + } return success; } 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 8002b9fa..f5de98b1 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 @@ -73,6 +73,8 @@ public abstract class AbstractSource extends BasedConfigurable implements ISourc */ protected int maxFetchLogGroupSize = 100; + protected List logFingerprintFields;//log fingerprint to filter msg quickly + /** * 数据源投递消息的算子,此算子用来接收source的数据,做处理 */ @@ -517,6 +519,14 @@ public void setCheckpointTime(long checkpointTime) { this.checkpointTime = checkpointTime; } + public List getLogFingerprintFields() { + return logFingerprintFields; + } + + public void setLogFingerprintFields(List logFingerprintFields) { + this.logFingerprintFields = logFingerprintFields; + } + @Override public long getCheckpointTime() { return checkpointTime; diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/HyperscanRegex.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/HyperscanRegex.java index ed37e1cb..6ed1a156 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/HyperscanRegex.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/HyperscanRegex.java @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.streams.common.optimization; +import com.gliwka.hyperscan.wrapper.CompileErrorException; import com.gliwka.hyperscan.wrapper.Database; import com.gliwka.hyperscan.wrapper.Expression; import com.gliwka.hyperscan.wrapper.ExpressionFlag; @@ -27,13 +28,19 @@ import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.rocketmq.streams.common.utils.StringUtil; public class HyperscanRegex { - protected List regexs = new ArrayList<>(); + protected List allRegexs = new ArrayList<>();//all registe regex + protected Database db; protected Scanner scanner; protected AtomicBoolean hasCompile = new AtomicBoolean(false); - protected List list = new ArrayList<>(); + protected List expressionContextList = new ArrayList<>(); + + protected List notSupportCompileExpression=new ArrayList<>();//can not comile expressions + protected List supportCompileExpression = new ArrayList<>();//all regex exclude not support compile + /** * 把多个表达式放到库里 @@ -41,9 +48,10 @@ public class HyperscanRegex { * @param regex */ public void addRegex(String regex, T context) { - list.add(context); - Expression expression = new Expression(regex, EnumSet.of(ExpressionFlag.UTF8, ExpressionFlag.CASELESS, ExpressionFlag.SINGLEMATCH), list.size() - 1); - regexs.add(expression); + expressionContextList.add(context); + Expression expression = new Expression(regex, EnumSet.of(ExpressionFlag.UTF8, ExpressionFlag.CASELESS, ExpressionFlag.SINGLEMATCH), expressionContextList.size() - 1); + allRegexs.add(expression); + supportCompileExpression.add(expression); db = null; scanner = null; hasCompile.set(false); @@ -53,19 +61,28 @@ public void addRegex(String regex, T context) { * 完成编译 */ public void compile() { - try { - if (hasCompile.compareAndSet(false, true) && regexs.size() > 0) { - Database db = Database.compile(regexs); + if (!hasCompile.compareAndSet(false, true) || supportCompileExpression.size() == 0) { + return ; + } + while (true){ + try { + if(supportCompileExpression.size()==0){ + break; + } + Database db = Database.compile(supportCompileExpression); Scanner scanner = new Scanner(); scanner.allocScratch(db); this.db = db; this.scanner = scanner; + break; + } catch (CompileErrorException e) { + Expression expression=e.getFailedExpression(); + this.supportCompileExpression.remove(expression); + this.notSupportCompileExpression.add(expression); } - - } catch (Exception e) { - System.out.println("can not support this regex " + e.getMessage()); } + } /** @@ -78,6 +95,9 @@ public boolean match(String content) { if (scanner == null || db == null || hasCompile.get() == false) { compile(); } + if(content==null){ + return false; + } List matches = scanner.scan(db, content); if (matches.size() > 0) { return true; @@ -96,14 +116,26 @@ public Set matchExpression(String content) { if (scanner == null || db == null || hasCompile.get() == false) { compile(); } + if(content==null){ + return new HashSet<>(); + } List matches = scanner.scan(db, content); Set fireExpressions = new HashSet<>(); - if (matches.size() == 0) { - return fireExpressions; + if(this.notSupportCompileExpression.size()>0){ + for(Expression expression:this.notSupportCompileExpression){ + String regex=expression.getExpression(); + boolean isMatch=StringUtil.matchRegexCaseInsensitive(content,regex); + if(isMatch){ + int index=expression.getId(); + fireExpressions.add(expressionContextList.get(index)); + } + } } - for (Match match : matches) { - Integer index = match.getMatchedExpression().getId(); - fireExpressions.add(list.get(index)); + if(matches.size()>0){ + for (Match match : matches) { + Integer index = match.getMatchedExpression().getId(); + fireExpressions.add(expressionContextList.get(index)); + } } return fireExpressions; } diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/LikeRegex.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/LikeRegex.java index d14b7b6e..10d7cd92 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/LikeRegex.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/optimization/LikeRegex.java @@ -20,6 +20,7 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.utils.StringUtil; /** * 可以用sql中的like表示正则,系统负责完成转化 @@ -27,12 +28,15 @@ public class LikeRegex { private static final Log LOG = LogFactory.getLog(LikeRegex.class); public static final String SPECAIL_WORD = "%"; - public static String[] regexSpecialWords = {"\\", "$", "(", ")", "*", "+", ".", "[", "]", "?", "^", "{", "}", "|"}; + public static String[] regexSpecialWords = {"\\(", "\\)", "\\*", "\\+", "\\.", "\\[", "\\]", "\\?", "\\^", "\\{", "\\}", "\\|"}; + //public static String[] regexSpecialWords = {"\\\\","\\$", "\\(", "\\)", "\\*", "\\+", "\\.", "\\[", "\\]", "\\?", "\\^", "\\{", "\\}", "\\|"}; protected String likeStr; protected boolean isStartFlag = true; protected boolean isEndFlag = true; + protected boolean hasUnderline=false; protected List quickMatchWord = new ArrayList<>(); + protected List specailWordIndex=new ArrayList<>(); public LikeRegex(String likeStr) { this.likeStr = likeStr; @@ -41,6 +45,9 @@ public LikeRegex(String likeStr) { public void parse() { String tmp = likeStr; + if(tmp.indexOf("_")!=-1){ + hasUnderline=true; + } if (tmp == null) { return; } @@ -59,6 +66,7 @@ public void parse() { String[] words = tmp.split(SPECAIL_WORD); for (int i = 0; i < words.length; i++) { quickMatchWord.add(words[i]); + specailWordIndex.add(i); } } @@ -66,6 +74,10 @@ public boolean match(String content) { if (content == null) { return false; } + if(hasUnderline){ + String regex=createRegex(); + return StringUtil.matchRegex(content,regex); + } if (quickMatchWord == null || quickMatchWord.size() == 0) { LOG.warn("like may be parse error, words is empty " + likeStr); return false; @@ -90,6 +102,54 @@ public boolean match(String content) { return true; } + public String createRegex(){ + StringBuilder regex=new StringBuilder(); + + boolean isFirst=true; + for(String word:this.quickMatchWord){ + if(isFirst){ + isFirst=false; + }else { + regex.append(".*"); + } + String subRegex=word; + + for(String regexSpecialWord:regexSpecialWords){ + try { + subRegex = subRegex.replaceAll(regexSpecialWord,"\\"+regexSpecialWord); + + }catch (Exception e){ + LOG.error(regexSpecialWord+" "+subRegex+"\r\n"+ e.getMessage(),e); + throw new RuntimeException(e); + } + } + + if (subRegex.indexOf("$")!=-1){ + String newSubRegex=""; + for(int i=0;i regexs=new ArrayList<>(); - //regexs.add(regex2Optimization); - //FileUtil.write("/Users/yuanxiaodong/Documents/dipper_engine/regex.txt",regexs,true); + //List allRegexs=new ArrayList<>(); + //allRegexs.add(regex2Optimization); + //FileUtil.write("/Users/yuanxiaodong/Documents/dipper_engine/regex.txt",allRegexs,true); } 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 00cee1a3..9310c0a7 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 @@ -537,6 +537,9 @@ public static T getBeanFieldOrJsonValue(Object bean, Class clazz, String fie Object result = null; Object modelBean = bean;// ChannelMessage for (String name : fieldNames) {// messageBody.type + if(modelBean==null){ + return null; + } if (JSONObject.class.isInstance(modelBean)) { result = getJsonItemValue(modelBean, clazz, name); } else if (JSONArray.class.isInstance(modelBean)) { 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 f5455bb6..860d0374 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 @@ -253,7 +253,7 @@ private boolean processExpress(Rule rule, RuleContext context) { if (rule.getExpressionName() == null) { return false; } - // rule.getGroupExpressionManager().matchAndSetResult(context,rule); + rule.getGroupExpressionManager().matchAndSetResult(context,rule); Expression expression = context.getExpression(rule.getExpressionName()); if (expression == null) { return false; 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 94907737..0bc574b3 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 @@ -125,8 +125,8 @@ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableS initVar(configurableService); initAction(configurableService); initMetaData(configurableService); - // this.optimize(); - // groupExpressionManager.compile(); + this.optimize(); + groupExpressionManager.compile(); } public void addAction(ChannelAction action) { 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 31e95a67..9d3223eb 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 @@ -29,16 +29,20 @@ import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.filter.context.RuleContext; +import org.apache.rocketmq.streams.filter.function.expression.LikeFunction; import org.apache.rocketmq.streams.filter.function.expression.RegexFunction; import org.apache.rocketmq.streams.filter.operator.Rule; +/** + * group by var name from all expression + */ public class GroupExpression extends Expression> { protected Rule rule; protected String varName; protected static IntValueKV cache = new IntValueKV(3000000); protected boolean isOrRelation = true;//是否是or关系 protected Map expressionName2Result = new HashMap<>();//正则类表达式的名字和结果的映射 - protected Set regeExpressionNameSet = new HashSet<>();//正则类表达式的名字 + protected Set regexExpressionNameSet = new HashSet<>();//正则类表达式的名字 public GroupExpression(Rule rule, String varName, boolean isOrRelation) { this.rule = rule; @@ -107,13 +111,16 @@ private Boolean executeMatch(RuleContext context, Rule rule) { public void addExpressionName(Expression expression) { if (RegexFunction.isRegex(expression.getFunctionName())) { - regeExpressionNameSet.add(expression.getConfigureName()); + regexExpressionNameSet.add(expression.getConfigureName()); + } + if(LikeFunction.isLikeFunciton(expression.getFunctionName())){ + regexExpressionNameSet.add(expression.getConfigureName()); } getValue().add(expression); } public void setRegexResult(Set allRegexResult) { - Iterator it = regeExpressionNameSet.iterator(); + Iterator it = regexExpressionNameSet.iterator(); while (it.hasNext()) { String name = it.next(); Boolean value = allRegexResult.contains(name); diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpressionManager.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpressionManager.java index 346992ec..a9aa278f 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpressionManager.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/expression/GroupExpressionManager.java @@ -26,7 +26,9 @@ import java.util.Map.Entry; import java.util.Set; import org.apache.rocketmq.streams.common.optimization.HyperscanRegex; +import org.apache.rocketmq.streams.common.optimization.LikeRegex; import org.apache.rocketmq.streams.filter.context.RuleContext; +import org.apache.rocketmq.streams.filter.function.expression.LikeFunction; import org.apache.rocketmq.streams.filter.function.expression.RegexFunction; import org.apache.rocketmq.streams.filter.operator.Rule; @@ -45,14 +47,23 @@ public GroupExpressionManager(Rule rule) { public void compile() { for (Expression expression : rule.getExpressionMap().values()) { - if (SimpleExpression.class.isInstance(expression) && RegexFunction.isRegex(expression.getFunctionName())) { + if (SimpleExpression.class.isInstance(expression) &&(RegexFunction.isRegex(expression.getFunctionName())|| LikeFunction.isLikeFunciton(expression.getFunctionName()))) { String varName = expression.getVarName(); HyperscanRegex hyperscanRegex = hyperscanRegexMap.get(varName); if (hyperscanRegex == null) { hyperscanRegex = new HyperscanRegex(); hyperscanRegexMap.put(varName, hyperscanRegex); } - hyperscanRegex.addRegex((String)expression.getValue(), expression.getConfigureName()); + if(LikeFunction.isLikeFunciton(expression.getFunctionName())){ + String like=(String)expression.getValue(); + LikeRegex likeRegex=new LikeRegex(like); + hyperscanRegex.addRegex(likeRegex.createRegex(),expression.getConfigureName()); + }else if(RegexFunction.isRegex(expression.getFunctionName())){ + hyperscanRegex.addRegex((String)expression.getValue(), expression.getConfigureName()); + }else { + throw new RuntimeException("can not support other function name "+ expression.getFunctionName()); + } + } } for (HyperscanRegex hyperscanRegex : hyperscanRegexMap.values()) { 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 7c0e02ef..ac7a949d 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 @@ -45,7 +45,10 @@ public class PiplineLogFingerprintAnalysis { public PiplineLogFingerprintAnalysis(ChainPipeline pipline) { this.pipline = pipline; } - + public PiplineLogFingerprintAnalysis(ChainPipeline pipline,int filterIndex) { + this.pipline = pipline; + this.filterIndex=filterIndex; + } /** * 通过分析pipline找到可以过滤的日志指纹,插入到合适的stage中。如最顶部的stage中 * diff --git a/rocketmq-streams-lease/src/test/java/org/apache/rocketmq/streams/lease/LeaseComponentTest.java b/rocketmq-streams-lease/src/test/java/org/apache/rocketmq/streams/lease/LeaseComponentTest.java index 8f9b8a11..bff40c10 100644 --- a/rocketmq-streams-lease/src/test/java/org/apache/rocketmq/streams/lease/LeaseComponentTest.java +++ b/rocketmq-streams-lease/src/test/java/org/apache/rocketmq/streams/lease/LeaseComponentTest.java @@ -49,8 +49,8 @@ public LeaseComponentTest() { @Test public void testLease() throws InterruptedException { - String leaseName = "lease.test"; - int leaseTime = 5; + String leaseName = "lease.test";//自己取个名字,和其他租约不同就行,可以用source 的namespace+configurableName + int leaseTime = 5;//租期,时间是秒。如果机器挂了, LeaseComponent.getInstance().getService().startLeaseTask(leaseName, leaseTime, new ILeaseGetCallback() { @Override public void callback(Date nextLeaseDate) { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonValueFunction.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonValueFunction.java index 5ae0e604..59291550 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonValueFunction.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/impl/json/JsonValueFunction.java @@ -65,6 +65,9 @@ public Object extra(IMessage message, FunctionContext context, } else { bean = JSON.parseObject(value); } + if(bean==null){ + return null; + } return ReflectUtil.getBeanFieldOrJsonValue(bean, pattern); } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java index 84f435ef..751bc9d7 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/expression/ScriptExpression.java @@ -31,10 +31,8 @@ import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.script.ScriptComponent; import org.apache.rocketmq.streams.script.context.FunctionContext; -import org.apache.rocketmq.streams.script.function.impl.string.RegexFunction; import org.apache.rocketmq.streams.script.function.model.FunctionConfigure; import org.apache.rocketmq.streams.script.optimization.compile.CompileScriptExpression; -import org.apache.rocketmq.streams.script.optimization.performance.ScriptExpressionGroupsProxy; import org.apache.rocketmq.streams.script.service.IScriptExpression; import org.apache.rocketmq.streams.script.service.IScriptParamter; import org.apache.rocketmq.streams.script.utils.FunctionUtils; @@ -70,37 +68,13 @@ public Object executeExpression(IMessage message, FunctionContext context) { FunctionUtils.getValue(message, context, value.toString())); return value; } - Object value = null; - - - if(RegexFunction.isRegexFunction(functionName)){ - ScriptParameter varParameter=(ScriptParameter) getParameters().get(0); - ScriptParameter regexParameter=(ScriptParameter) getParameters().get(1); - String varName=getParameterValue(varParameter); - String expression=getParameterValue(regexParameter); - if(varName!=null||expression!=null){ - value= ScriptExpressionGroupsProxy.inFilterCache(varName,expression,message,context); - } - - - } - if(value!=null){ - if (StringUtil.isNotEmpty(newFieldName) && value != null) { - setValue2Var(message, context, newFieldName, value); - } - return value; - } if (compileScriptExpression != null) { value = compileScriptExpression.execute(message, context); } else { value = execute(message, context); } - if (StringUtil.isNotEmpty(newFieldName) && value != null) { - setValue2Var(message, context, newFieldName, value); - } - //monitor.setResult(value); //monitor.endMonitor(); //if (monitor.isSlow()) { @@ -122,17 +96,6 @@ public Object executeExpression(IMessage message, FunctionContext context) { } - protected String getParameterValue(IScriptParamter scriptParamter) { - if (!ScriptParameter.class.isInstance(scriptParamter)) { - return null; - } - ScriptParameter parameter = (ScriptParameter)scriptParamter; - if (parameter.getRigthVarName() != null) { - return null; - } - return FunctionUtils.getConstant(parameter.getLeftVarName()); - } - private ScriptComponent scriptComponent = ScriptComponent.getInstance(); public Object execute(IMessage message, FunctionContext context) { @@ -161,7 +124,10 @@ public Object execute(IMessage message, FunctionContext context) { } Object value = functionConfigure.execute(ps); compileScriptExpression = new CompileScriptExpression(this, functionConfigure); - + if (StringUtil.isNotEmpty(newFieldName) && value != null) { + setValue2Var(message, context, newFieldName, value); + //message.getMessageBody().put(newFieldName, value); + } return value; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/impl/ScriptServiceImpl.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/impl/ScriptServiceImpl.java index f88fb5d3..fea38717 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/impl/ScriptServiceImpl.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/impl/ScriptServiceImpl.java @@ -46,6 +46,9 @@ public ScriptServiceImpl() { @Override public List executeScript(IMessage message, FunctionContext context, AbstractScript, FunctionContext> script) { script.doMessage(message, context); + if(context.isSplitModel()){ + return context.getSplitMessages(); + } List messages = new ArrayList<>(); messages.add(message); return messages; diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDAFScript.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDAFScript.java index ae471a10..7089b437 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDAFScript.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/service/udf/UDAFScript.java @@ -66,10 +66,6 @@ public void registFunctionSerivce(IFunctionService iFunctionService) { } } - @Override - protected boolean initConfigurable() { - return true; - } @Override protected boolean initBeanClass(IFunctionService iFunctionService) { 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 a75d300c..2cf82456 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 @@ -88,7 +88,7 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo /** * 用消息中的哪个字段做时间字段 */ - protected String timeFieldName; + protected String timeFieldName ; /** * having column in having clause eg: key:'having_sum_0001' value:'having_sum_0001=SUM(OrderPrice)<2000' note: here ignore the logical relation value may be multi expression which split by ${SCRIPT_SPLIT_CHAR} update: change sql(move the function into select clause) to escape function in having clause @@ -103,7 +103,7 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo /** * SQL中group by的字段,使用;拼接,如"name;age" */ - protected String groupByFieldName; + protected String groupByFieldName ; /** * 意义同blink中,允许最晚的消息到达时间,单位是分钟 @@ -121,7 +121,7 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo /** * 主要是做兼容,以前设计的窗口时间是分钟为单位,如果有秒作为窗口时间的,通过设置timeUntiAdjust=1来实现。 后续需要调整成直接秒级窗口 */ - protected int timeUnitAdjust = 60; + protected int timeUnitAdjust=60; /** * the variable name of window size which can be got from message */ @@ -145,14 +145,14 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo /** * 默认为空,窗口的触发类似flink,在测试模式下,因为消息有界,期望当消息发送完成后能触发,可以设置两条消息的最大间隔,超过这个间隔,将直接触发消息 */ - protected Long msgMaxGapSecond = 10L; + protected Long msgMaxGapSecond=10L; /** * 是否支持过期数据的计算 过期:当前时间大于数据所在窗口的触发时间 */ - protected int fireMode = 0;//0:普通触发,firetime后收到数据丢弃;1:多实例多次独立触发,在watermark时间内,同starttime,endtime创建多个实例,多次触发;2.单实例,多次独立触发,每次触发是最新值 + protected int fireMode=0;//0:普通触发,firetime后收到数据丢弃;1:多实例多次独立触发,在watermark时间内,同starttime,endtime创建多个实例,多次触发;2.单实例,多次独立触发,每次触发是最新值 - protected boolean isLocalStorageOnly = true;//是否只用本地存储,可以提高性能,但不保证可靠性 + protected boolean isLocalStorageOnly=true;//是否只用本地存储,可以提高性能,但不保证可靠性 protected String reduceSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码 protected transient IReducer reducer; /** @@ -206,12 +206,12 @@ protected boolean initConfigurable() { /** * 如果没有db配置,不开启远程存储服务 */ - if (!ORMUtil.hasConfigueDB()) { - isLocalStorageOnly = true; + if(!ORMUtil.hasConfigueDB()){ + isLocalStorageOnly=true; } - sqlCache = new SQLCache(isLocalStorageOnly); - AbstractWindow window = this; - windowCache = new WindowCache() { + sqlCache=new SQLCache(isLocalStorageOnly); + AbstractWindow window=this; + windowCache=new WindowCache(){ @Override protected String generateShuffleKey(IMessage message) { @@ -227,12 +227,12 @@ protected String generateShuffleKey(IMessage message) { initFunctionExecutor(); //启动shuffle channel 实现消息shuffle以及接收shuffle消息并处理 // FireManager.getInstance().startFireCheck(); - if (StringUtil.isNotEmpty(this.reduceSerializeValue)) { - byte[] bytes = Base64Utils.decode(this.reduceSerializeValue); + if(StringUtil.isNotEmpty(this.reduceSerializeValue)){ + byte[] bytes= Base64Utils.decode( this.reduceSerializeValue); reducer = InstantiationUtil.deserializeObject(bytes); } - eventTimeManager = new EventTimeManager(); - windowMaxValueManager = new WindowMaxValueManager(this, sqlCache); + eventTimeManager=new EventTimeManager(); + windowMaxValueManager = new WindowMaxValueManager(this,sqlCache); return success; } @@ -283,22 +283,22 @@ public String createWindowInstance(String startTime,String endTime,String fireTi } */ - public WindowInstance createWindowInstance(String startTime, String endTime, String fireTime, String splitId) { - WindowInstance windowInstance = new WindowInstance(); + public WindowInstance createWindowInstance(String startTime, String endTime, String fireTime,String splitId) { + WindowInstance windowInstance =new WindowInstance(); windowInstance.setFireTime(fireTime); windowInstance.setStartTime(startTime); windowInstance.setEndTime(endTime); windowInstance.setSplitId(splitId); windowInstance.setGmtCreate(new Date()); windowInstance.setGmtModified(new Date()); - windowInstance.setWindowInstanceName(createWindowInstanceName(startTime, endTime, fireTime)); + windowInstance.setWindowInstanceName(createWindowInstanceName(startTime,endTime,fireTime)); windowInstance.setWindowName(getConfigureName()); windowInstance.setWindowNameSpace(getNameSpace()); - String windowInstanceId = windowInstance.createWindowInstanceId(); + String windowInstanceId =windowInstance.createWindowInstanceId(); String dbWindowInstanceId = StringUtil.createMD5Str(windowInstanceId); windowInstance.setWindowInstanceKey(dbWindowInstanceId); - windowInstance.setWindowInstanceSplitName(StringUtil.createMD5Str(MapKeyUtil.createKey(getNameSpace(), getConfigureName(), splitId))); + windowInstance.setWindowInstanceSplitName(StringUtil.createMD5Str(MapKeyUtil.createKey(getNameSpace(), getConfigureName(),splitId))); windowInstance.setNewWindowInstance(true); return windowInstance; } @@ -311,8 +311,8 @@ public WindowInstance createWindowInstance(String startTime, String endTime, Str * @param fireTime * @return */ - public String createWindowInstanceName(String startTime, String endTime, String fireTime) { - return fireMode == 0 ? getConfigureName() : fireTime; + public String createWindowInstanceName(String startTime, String endTime, String fireTime){ + return fireMode==0?getConfigureName():fireTime; } /** @@ -323,14 +323,14 @@ public String createWindowInstanceName(String startTime, String endTime, String * @return */ - public long incrementAndGetSplitNumber(WindowInstance instance, String shuffleId) { - long maxValue = windowMaxValueManager.incrementAndGetSplitNumber(instance, shuffleId); + public long incrementAndGetSplitNumber(WindowInstance instance,String shuffleId){ + long maxValue= windowMaxValueManager.incrementAndGetSplitNumber(instance,shuffleId); return maxValue; } public abstract Class getWindowBaseValueClass(); - public abstract int fireWindowInstance(WindowInstance windowInstance, Map queueId2Offset); + public abstract int fireWindowInstance(WindowInstance windowInstance,MapqueueId2Offset) ; /** * 计算每条记录的group by值,对于groupby分组,里面任何字段不能为null值,如果为null值,这条记录会被忽略 @@ -338,21 +338,21 @@ public long incrementAndGetSplitNumber(WindowInstance instance, String shuffleId * @param message * @return */ - protected String generateShuffleKey(IMessage message) { + protected String generateShuffleKey(IMessage message){ if (StringUtil.isEmpty(groupByFieldName)) { return null; } - JSONObject msg = message.getMessageBody(); + JSONObject msg=message.getMessageBody(); String[] fieldNames = groupByFieldName.split(";"); - String[] values = new String[fieldNames.length]; + String[] values=new String[fieldNames.length]; boolean isFirst = true; - int i = 0; + int i=0; for (String filedName : fieldNames) { if (isFirst) { isFirst = false; } String value = msg.getString(filedName); - values[i] = value; + values[i]=value; i++; } return MapKeyUtil.createKey(values); @@ -360,8 +360,8 @@ protected String generateShuffleKey(IMessage message) { public abstract void clearFireWindowInstance(WindowInstance windowInstance); - public void clearFire(WindowInstance windowInstance) { - if (windowInstance == null) { + public void clearFire(WindowInstance windowInstance){ + if(windowInstance==null){ return; } clearFireWindowInstance(windowInstance); @@ -413,7 +413,7 @@ protected void initFunctionExecutor() { scriptBuilder = new StringBuilder(); } String[] functionParameterNames = scriptParameterList.stream().map( - scriptParameter -> scriptParameter.getScriptParameterStr()).collect(Collectors.toList()) + scriptParameter -> scriptParameter.getScriptParameterStr()).collect(Collectors.toList()) .toArray(new String[0]); AggregationScript accEngine = new AggregationScript( ((ScriptExpression)expression).getNewFieldName(), functionName, @@ -461,8 +461,8 @@ protected void initFunctionExecutor() { * @param message * @return */ - public List queryOrCreateWindowInstance(IMessage message, String queueId) { - return WindowInstance.getOrCreateWindowInstance(this, WindowInstance.getOccurTime(this, message), timeUnitAdjust, + public List queryOrCreateWindowInstance(IMessage message,String queueId) { + return WindowInstance.getOrCreateWindowInstance(this, WindowInstance.getOccurTime(this, message), timeUnitAdjust, queueId); } @@ -472,8 +472,8 @@ public List queryOrCreateWindowInstance(IMessage message, String * @param msg * @return */ - public void updateMaxEventTime(IMessage msg) { - eventTimeManager.updateEventTime(msg, this); + public void updateMaxEventTime(IMessage msg){ + eventTimeManager.updateEventTime(msg,this); } public Long getMaxEventTime(String queueId) { @@ -485,15 +485,15 @@ public Long getMaxEventTime(String queueId) { * * @param windowValueList */ - public void sendFireMessage(List windowValueList, String queueId) { + public void sendFireMessage(List windowValueList,String queueId) { int count = 0; - List msgs = new ArrayList<>(); + List msgs=new ArrayList<>(); for (WindowValue windowValue : windowValueList) { JSONObject message = new JSONObject(); - if (JSONObject.class.isInstance(windowValue.getcomputedResult())) { - message = (JSONObject)windowValue.getcomputedResult(); - } else { + if(JSONObject.class.isInstance(windowValue.getcomputedResult())){ + message=(JSONObject)windowValue.getcomputedResult(); + }else { Iterator> it = windowValue.iteratorComputedColumnResult(); while (it.hasNext()) { Entry entry = it.next(); @@ -501,23 +501,23 @@ public void sendFireMessage(List windowValueList, String queueId) { } } - Long fireTime = DateUtil.parseTime(windowValue.getFireTime()).getTime(); - long baseTime = 1577808000000L;//set base time from 2021-01-01 00:00:00 - int sameFireCount = 0; - if (fireMode != 0) { - Long endTime = DateUtil.parseTime(windowValue.getEndTime()).getTime(); - sameFireCount = (int)((fireTime - endTime) / 1000) / sizeInterval * timeUnitAdjust; - if (sameFireCount >= 1) { - sameFireCount = 1; + Long fireTime=DateUtil.parseTime(windowValue.getFireTime()).getTime(); + long baseTime= 1577808000000L ;//set base time from 2021-01-01 00:00:00 + int sameFireCount=0; + if(fireMode!=0){ + Long endTime=DateUtil.parseTime(windowValue.getEndTime()).getTime(); + sameFireCount=(int)((fireTime-endTime)/1000)/sizeInterval*timeUnitAdjust; + if(sameFireCount>=1){ + sameFireCount=1; } } //can keep offset in order - Long offset = ((fireTime - baseTime) / 1000 * 10 + sameFireCount) * 100000000 + windowValue.getPartitionNum(); - message.put("windowInstanceId", windowValue.getWindowInstancePartitionId()); - message.put("start_time", windowValue.getStartTime()); - message.put("end_time", windowValue.getEndTime()); - message.put("offset", offset); - Message newMessage = windowFireSource.createMessage(message, queueId, offset + "", false); + Long offset=((fireTime-baseTime)/1000*10+sameFireCount)*100000000+windowValue.getPartitionNum(); + message.put("windowInstanceId",windowValue.getWindowInstancePartitionId()); + message.put("start_time",windowValue.getStartTime()); + message.put("end_time",windowValue.getEndTime()); + message.put("offset",offset); + Message newMessage=windowFireSource.createMessage(message,queueId,offset+"",false); newMessage.getHeader().setOffsetIsLong(true); if (count == windowValueList.size() - 1) { newMessage.getHeader().setNeedFlush(true); @@ -528,8 +528,8 @@ public void sendFireMessage(List windowValueList, String queueId) { count++; } - if (DebugWriter.getDebugWriter(this.getConfigureName()).isOpenDebug()) { - DebugWriter.getDebugWriter(this.getConfigureName()).writeWindowFire(this, msgs, queueId); + if(DebugWriter.getDebugWriter(this.getConfigureName()).isOpenDebug()){ + DebugWriter.getDebugWriter(this.getConfigureName()).writeWindowFire(this,msgs,queueId); } } @@ -689,7 +689,7 @@ public IReducer getReducer() { public void setReducer(IReducer reducer) { this.reducer = reducer; byte[] bytes = InstantiationUtil.serializeObject(reducer); - this.reduceSerializeValue = Base64Utils.encode(bytes); + this.reduceSerializeValue=Base64Utils.encode(bytes); } public int getTimeUnitAdjust() { @@ -754,8 +754,8 @@ public SQLCache getSqlCache() { return sqlCache; } - public void initWindowInstanceMaxSplitNum(WindowInstance instance) { - getWindowMaxValueManager().initMaxSplitNum(instance, queryWindowInstanceMaxSplitNum(instance)); + public void initWindowInstanceMaxSplitNum(WindowInstance instance){ + getWindowMaxValueManager().initMaxSplitNum(instance,queryWindowInstanceMaxSplitNum(instance)); } protected abstract Long queryWindowInstanceMaxSplitNum(WindowInstance instance); diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java index 4bcfe3a0..ef60fe33 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java @@ -35,9 +35,9 @@ */ public class SQLCache extends AbstractMultiSplitMessageCache { - protected Boolean isOpenCache = true;//if false,then execute sql when receive sql - protected Set firedWindowInstances = new HashSet<>();//fired window instance ,if the owned sqls have not commit, can cancel the sqls - protected Map windowInstance2Index = new HashMap<>();//set index to ISQLElement group by window instance + protected Boolean isOpenCache=true;//if false,then execute sql when receive sql + protected Set firedWindowInstances=new HashSet<>();//fired window instance ,if the owned sqls have not commit, can cancel the sqls + protected Map windowInstance2Index=new HashMap<>();//set index to ISQLElement group by window instance protected boolean isLocalOnly;