diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java index eb91af6c..2df207a5 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/WindowStream.java @@ -59,6 +59,53 @@ public WindowStream count(String asName) { return this; } + /** + * distinct算子 + * + * @param fieldName + * @param asName + * @return + */ + public WindowStream distinct(String fieldName, String asName) { + window.getSelectMap().put(asName, asName + "=distinct(" + fieldName + ")"); + return this; + } + + /** + * count_distinct算子 + * + * @param fieldName + * @param asName + * @return + */ + public WindowStream count_distinct(String fieldName, String asName) { + String distinctName = "__" + fieldName + "_distinct_" + asName + "__"; + String prefix = distinctName + "=distinct(" + fieldName + ")"; + String suffix = asName + "=count(" + distinctName + ")"; + window.getSelectMap().put(asName, prefix + ";" + suffix); + return this; + } + + public WindowStream count_distinct_2(String fieldName, String asName) { + String distinctName = "__" + fieldName + "_distinct_" + asName + "__"; + String prefix = distinctName + "=distinct2(" + fieldName + ",HIT_WINDOW_INSTANCE_ID,SHUFFLE_KEY)"; + String suffix = asName + "=count(" + distinctName + ")"; + window.getSelectMap().put(asName, prefix + ";" + suffix); + return this; + } + + /** + * count_distinct算子(数据量大,容忍较少错误率) + * + * @param fieldName + * @param asName + * @return + */ + public WindowStream count_distinct_large(String fieldName, String asName) { + window.getSelectMap().put(asName, asName + "=count_distinct(" + fieldName + ")"); + return this; + } + /** * 做min算子 * diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/SessionWindow.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/SessionWindow.java index 67c7f226..98fe719b 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/SessionWindow.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/SessionWindow.java @@ -20,13 +20,22 @@ public class SessionWindow { /** * 滑动窗口信息 + * * @param time * @return */ - public static WindowInfo of(Time time){ - WindowInfo windowInfo=new WindowInfo(); + public static WindowInfo of(Time time) { + WindowInfo windowInfo = new WindowInfo(); windowInfo.setType(WindowInfo.SESSION_WINDOW); - windowInfo.setWindowSize(time); + windowInfo.setSessionTimeout(time); + return windowInfo; + } + + public static WindowInfo of(Time time, String timeField) { + WindowInfo windowInfo = new WindowInfo(); + windowInfo.setType(WindowInfo.SESSION_WINDOW); + windowInfo.setSessionTimeout(time); + windowInfo.setTimeField(timeField); return windowInfo; } } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/TumblingWindow.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/TumblingWindow.java index 62c0ce31..1ff0a806 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/TumblingWindow.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/TumblingWindow.java @@ -30,4 +30,12 @@ public static WindowInfo of(Time time) { windowInfo.setWindowSize(time); return windowInfo; } + + public static WindowInfo of(Time time, String timeField) { + WindowInfo windowInfo = new WindowInfo(); + windowInfo.setType(WindowInfo.TUMBLING_WINDOW); + windowInfo.setWindowSize(time); + windowInfo.setTimeField(timeField); + return windowInfo; + } } diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/WindowInfo.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/WindowInfo.java index 64be503c..39ff366a 100644 --- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/WindowInfo.java +++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/window/WindowInfo.java @@ -17,7 +17,7 @@ package org.apache.rocketmq.streams.client.transform.window; import org.apache.rocketmq.streams.window.operator.AbstractWindow; -import org.apache.rocketmq.streams.window.operator.impl.SessionWindow; +import org.apache.rocketmq.streams.window.operator.impl.SessionOperator; import org.apache.rocketmq.streams.window.operator.impl.WindowOperator; /** @@ -26,10 +26,16 @@ public class WindowInfo { public static int HOPPING_WINDOW = 1;//滑动窗口 public static int TUMBLING_WINDOW = 2;//滚动窗口 - public static int SESSION_WINDOW = 23; + public static int SESSION_WINDOW = 3; protected int type;//window类型 hopping,Tumbling protected Time windowSize;//窗口大小 protected Time windowSlide;//滑动大小 + /** + * 会话窗口的超时时间 + */ + protected Time sessionTimeout; + + protected String timeField; /** * 创建窗口 @@ -48,12 +54,11 @@ public AbstractWindow createWindow() { window.setTimeUnitAdjust(1); window.setSizeInterval(windowSize.getValue()); } else if (type == SESSION_WINDOW) { - window = new SessionWindow(); - window.setTimeUnitAdjust(1); - window.setSizeInterval(windowSize.getValue()); + window = new SessionOperator(sessionTimeout.getValue()); } else { throw new RuntimeException("can not support the type ,expect 1,2,3。actual is " + type); } + window.setTimeFieldName(timeField); return window; } @@ -80,4 +85,20 @@ public Time getWindowSlide() { public void setWindowSlide(Time windowSlide) { this.windowSlide = windowSlide; } + + public Time getSessionTimeout() { + return sessionTimeout; + } + + public void setSessionTimeout(Time sessionTimeout) { + this.sessionTimeout = sessionTimeout; + } + + public String getTimeField() { + return timeField; + } + + public void setTimeField(String timeField) { + this.timeField = timeField; + } } diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java index 25eed73d..164fa8ef 100644 --- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java +++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/WindowTest.java @@ -18,11 +18,21 @@ package org.apache.rocketmq.streams.client; import com.alibaba.fastjson.JSONObject; +import java.io.File; import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.rocketmq.streams.client.transform.window.SessionWindow; import org.apache.rocketmq.streams.client.transform.window.Time; import org.apache.rocketmq.streams.client.transform.window.TumblingWindow; import org.apache.rocketmq.streams.common.functions.ForEachFunction; import org.apache.rocketmq.streams.common.functions.MapFunction; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.junit.Assert; import org.junit.Test; public class WindowTest implements Serializable { @@ -31,7 +41,7 @@ public class WindowTest implements Serializable { public void testWindow() { StreamBuilder.dataStream("namespace", "name") .fromFile("/Users/duheng/project/opensource/sls_100.txt", false) - .map((MapFunction)message -> JSONObject.parseObject(message)) + .map((MapFunction) message -> JSONObject.parseObject(message)) .window(TumblingWindow.of(Time.seconds(5))) .groupBy("ProjectName", "LogStore") .setLocalStorageOnly(true) @@ -81,4 +91,196 @@ public void foreach(JSONObject o) { // }).toPrint().start(); // } + @Test + public void testSession() { + //dataset + List behaviorList = new ArrayList<>(); + + JSONObject userA = new JSONObject(); + userA.put("time", DateUtil.parse("2021-09-09 10:00:00")); + userA.put("user", "userA"); + userA.put("movie", "movie001"); + userA.put("flag", 1); + behaviorList.add(userA.toJSONString()); + + userA = new JSONObject(); + userA.put("time", DateUtil.parse("2021-09-09 10:00:01")); + userA.put("user", "userA"); + userA.put("movie", "movie002"); + userA.put("flag", 1); + behaviorList.add(userA.toJSONString()); + + JSONObject userB = new JSONObject(); + userB.put("time", DateUtil.parse("2021-09-09 10:00:00")); + userB.put("user", "userB"); + userB.put("movie", "movie003"); + userB.put("flag", 1); + behaviorList.add(userB.toJSONString()); + + JSONObject userC = new JSONObject(); + userC.put("time", DateUtil.parse("2021-09-09 10:00:00")); + userC.put("user", "userC"); + userC.put("movie", "movie004"); + userC.put("flag", 1); + behaviorList.add(userC.toJSONString()); + + userC = new JSONObject(); + userC.put("time", DateUtil.parse("2021-09-09 10:00:06")); + userC.put("user", "userC"); + userC.put("movie", "movie005"); + userC.put("flag", 1); + behaviorList.add(userC.toJSONString()); + + File dataFile = null; + try { + dataFile = File.createTempFile("behavior", ".txt"); + FileUtils.writeLines(dataFile, behaviorList); + } catch (Exception e) { + e.printStackTrace(); + } + + File resultFile = null; + try { + resultFile = File.createTempFile("behavior.txt", ".session"); + } catch (Exception e) { + e.printStackTrace(); + } + + StreamBuilder.dataStream("namespace", "session_test") + .fromFile(dataFile.getAbsolutePath(), false) + .map((MapFunction) message -> JSONObject.parseObject(message)) + .window(SessionWindow.of(Time.seconds(5), "time")) + .groupBy("user") + .setLocalStorageOnly(true) + .sum("flag", "count") + .toDataSteam() + .toFile(resultFile.getAbsolutePath()).start(true); + + try { + Thread.sleep(1 * 60 * 1000); + List sessionList = FileUtils.readLines(resultFile, "UTF-8"); + Map, Integer>>> sessionMap = new HashMap<>(4); + for (String line : sessionList) { + JSONObject object = JSONObject.parseObject(line); + String user = object.getString("user"); + String startTime = object.getString("start_time"); + String endTime = object.getString("end_time"); + Integer value = object.getIntValue("count"); + if (!sessionMap.containsKey(user)) { + sessionMap.put(user, new ArrayList<>()); + } + sessionMap.get(user).add(Pair.of(Pair.of(startTime, endTime), value)); + } + Assert.assertEquals(3, sessionMap.size()); + Assert.assertEquals(1, sessionMap.get("userA").size()); + Assert.assertEquals("2021-09-09 10:00:06", sessionMap.get("userA").get(0).getLeft().getRight()); + Assert.assertEquals(2, sessionMap.get("userC").size()); + Assert.assertEquals("2021-09-09 10:00:05", sessionMap.get("userC").get(0).getLeft().getRight()); + Assert.assertEquals("2021-09-09 10:00:06", sessionMap.get("userC").get(1).getLeft().getLeft()); + Assert.assertEquals(1, sessionMap.get("userB").size()); + + } catch (Exception e) { + e.printStackTrace(); + } finally { + dataFile.deleteOnExit(); + resultFile.deleteOnExit(); + } + + } + + @Test + public void testCountDistinct() { + //dataset + List behaviorList = new ArrayList<>(); + + JSONObject userA = new JSONObject(); + userA.put("time", DateUtil.parse("2021-09-09 10:00:00")); + userA.put("user", "userA"); + userA.put("page", "alibaba-inc.com"); + behaviorList.add(userA.toJSONString()); + + userA = new JSONObject(); + userA.put("time", DateUtil.parse("2021-09-09 10:01:00")); + userA.put("user", "userA"); + userA.put("page", "sina.com"); + behaviorList.add(userA.toJSONString()); + + userA = new JSONObject(); + userA.put("time", DateUtil.parse("2021-09-09 10:03:00")); + userA.put("user", "userA"); + userA.put("page", "alibaba-inc.com"); + behaviorList.add(userA.toJSONString()); + + JSONObject userB = new JSONObject(); + userB.put("time", DateUtil.parse("2021-09-09 10:00:00")); + userB.put("user", "userB"); + userB.put("page", "sohu.com"); + behaviorList.add(userB.toJSONString()); + + JSONObject userC = new JSONObject(); + userC.put("time", DateUtil.parse("2021-09-09 10:00:00")); + userC.put("user", "userC"); + userC.put("page", "qq.com"); + behaviorList.add(userC.toJSONString()); + + userC = new JSONObject(); + userC.put("time", DateUtil.parse("2021-09-09 10:03:06")); + userC.put("user", "userC"); + userC.put("page", "qq.com"); + behaviorList.add(userC.toJSONString()); + + File dataFile = null; + try { + dataFile = File.createTempFile("behavior", ".txt"); + FileUtils.writeLines(dataFile, behaviorList); + } catch (Exception e) { + e.printStackTrace(); + } + + File resultFile = null; + try { + resultFile = File.createTempFile("behavior.txt", ".session"); + } catch (Exception e) { + e.printStackTrace(); + } + + StreamBuilder.dataStream("namespace", "count_distinct_test") + .fromFile(dataFile.getAbsolutePath(), false) + .map((MapFunction) message -> JSONObject.parseObject(message)) + .window(TumblingWindow.of(Time.minutes(5), "time")) + .groupBy("user") + .setLocalStorageOnly(true) + .count_distinct("page", "uv") + .count_distinct_large("page", "uv_large") + .count_distinct_2("page","uv_2") + .toDataSteam() + .toFile(resultFile.getAbsolutePath()).start(true); + + try { + Thread.sleep(6 * 60 * 1000); + List sessionList = FileUtils.readLines(resultFile, "UTF-8"); + Map statisticMap = new HashMap<>(4); + for (String line : sessionList) { + JSONObject object = JSONObject.parseObject(line); + String user = object.getString("user"); + Integer userVisitCount = object.getInteger("uv"); + Integer userVisitCountBasedRocksDB = object.getInteger("uv_2"); + Integer userVisitCountLarge = object.getInteger("uv_large"); + Assert.assertEquals(userVisitCount, userVisitCountLarge); + Assert.assertEquals(userVisitCount, userVisitCountBasedRocksDB); + statisticMap.put(user, userVisitCount); + } + Assert.assertEquals(3, statisticMap.size()); + Assert.assertEquals(2, statisticMap.get("userA").longValue()); + Assert.assertEquals(1, statisticMap.get("userB").longValue()); + Assert.assertEquals(1, statisticMap.get("userC").longValue()); + + } catch (Exception e) { + e.printStackTrace(); + } finally { + dataFile.deleteOnExit(); + resultFile.deleteOnExit(); + } + } + } diff --git a/rocketmq-streams-commons/pom.xml b/rocketmq-streams-commons/pom.xml index 9ca587ff..231531bd 100755 --- a/rocketmq-streams-commons/pom.xml +++ b/rocketmq-streams-commons/pom.xml @@ -37,6 +37,13 @@ 3.11 + + + net.agkn + hll + 1.6.0 + + junit 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 aa7c1bc0..a3fdee71 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 @@ -161,6 +161,10 @@ public boolean flushMessage(List messages) { String queueId = message.getHeader().getQueueId(); MessageOffset messageOffset = message.getHeader().getMessageOffset(); ISource source = message.getHeader().getSource(); + //TODO why null? + if (source == null) { + continue; + } String pipelineName = message.getHeader().getPiplineName(); String sourceName = CheckPointManager.createSourceName(source, pipelineName); SourceState sourceState = this.sourceName2State.get(sourceName); diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/HllDataType.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/HllDataType.java new file mode 100644 index 00000000..df5055f8 --- /dev/null +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/datatype/HllDataType.java @@ -0,0 +1,63 @@ +package org.apache.rocketmq.streams.common.datatype; + +import com.alibaba.fastjson.JSONObject; +import net.agkn.hll.HLL; +import org.apache.rocketmq.streams.common.utils.Base64Utils; +import org.apache.rocketmq.streams.common.utils.StringUtil; + +/** + * @author arthur.liang + */ +public class HllDataType extends BaseDataType { + + public HllDataType(Class clazz) { + setDataClazz(clazz); + } + + public HllDataType() { + setDataClazz(HLL.class); + } + + @Override protected void setFieldValueToJson(JSONObject jsonObject) { + } + + @Override protected void setFieldValueFromJson(JSONObject jsonObject) { + + } + + @Override public DataType create() { + return this; + } + + @Override public String getDataTypeName() { + return HLL.class.getSimpleName(); + } + + @Override public boolean matchClass(Class clazz) { + return HLL.class.getSimpleName().equals(clazz.getSimpleName()); + } + + @Override public String toDataJson(HLL value) { + if (value != null) { + return Base64Utils.encode(value.toBytes()); + } + return null; + } + + @Override public HLL getData(String jsonValue) { + if (StringUtil.isNotEmpty(jsonValue)) { + return HLL.fromBytes(Base64Utils.decode(jsonValue)); + } + return null; + } + + public static void main(String[] args) { + HLL hll = new HLL(30, 8); + hll.addRaw(123456); + HllDataType dataType = new HllDataType(); + String content = dataType.toDataJson(hll); + HLL copy = dataType.getData(content); + System.out.println(copy.cardinality()); + } + +} diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IWindow.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IWindow.java index 66b05eed..86d30d29 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IWindow.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/topology/model/IWindow.java @@ -50,6 +50,9 @@ public interface IWindow */ Integer DEFAULT_WINDOW_SLIDE = 1; + + Integer DEFAULT_WINDOW_SESSION_TIMEOUT = 10; + // protected transient ExecutorService deleteService = Executors.newSingleThreadExecutor(); /** @@ -72,6 +75,8 @@ public interface IWindow */ String HOP_WINDOW = "hop"; + String SESSION_WINDOW = "session"; + /** * hop window type */ @@ -80,6 +85,11 @@ public interface IWindow int FIRE_DELAY_SECOND = 1;//触发延迟 + /** + * the delay time of system (ms) + */ + Integer SYS_DELAY_TIME = 3000; + /** * 窗口触发后,需要执行的逻辑 * diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java index 6e8967b8..94e6982a 100644 --- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java +++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/DataTypeUtil.java @@ -41,6 +41,7 @@ import org.apache.rocketmq.streams.common.datatype.DoubleDataType; import org.apache.rocketmq.streams.common.datatype.FloatDataType; import org.apache.rocketmq.streams.common.datatype.GenericParameterDataType; +import org.apache.rocketmq.streams.common.datatype.HllDataType; import org.apache.rocketmq.streams.common.datatype.IJsonable; import org.apache.rocketmq.streams.common.datatype.IntDataType; import org.apache.rocketmq.streams.common.datatype.JavaBeanDataType; @@ -69,6 +70,8 @@ public class DataTypeUtil { private static final Map typeCode2DataType = new HashMap<>(); static { + register(new HllDataType()); + register(new NumberDataType()); register(new StringDataType()); register(new IntDataType()); 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 fe8a3bbb..3de8aca1 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 @@ -23,6 +23,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.commons.lang3.tuple.Pair; import org.apache.rocketmq.streams.common.configurable.BasedConfigurable; import org.apache.rocketmq.streams.common.configurable.IConfigurable; import org.apache.rocketmq.streams.common.datatype.DataType; @@ -392,6 +393,27 @@ public static String createInSql(Collection collection) { return createInSql(values); } + /** + * create multi like sentences + * @param keywordList + * @return + */ + public static String createLikeSql(List> keywordList) { + if (CollectionUtil.isEmpty(keywordList)) { + return ""; + } + StringBuffer buffer = new StringBuffer(); + buffer.append(" "); + for (int index = 0; index < keywordList.size(); index++) { + Pair pair = keywordList.get(index); + buffer.append(pair.getKey() + " like '" + pair.getValue() + "'"); + if (index != (keywordList.size() - 1)) { + buffer.append(" or "); + } + } + return buffer.toString(); + } + public static String createInSql(String... values) { return createInSql(true, values); } diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexFunction.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexFunction.java index 11bd7f9e..2996ae71 100644 --- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexFunction.java +++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/function/expression/RegexFunction.java @@ -78,8 +78,8 @@ public Boolean doExpressionFunction(Expression expression, RuleContext context, String varString = ""; String regex = ""; - varString = String.valueOf(varObject).trim(); - regex = String.valueOf(valueObject).trim(); + varString = String.valueOf(varObject); + regex = String.valueOf(valueObject); Boolean cacheResult=context.getFilterCache(regex,varString); if(cacheResult!=null){ diff --git a/rocketmq-streams-script/pom.xml b/rocketmq-streams-script/pom.xml index 2f0e1185..a962be4d 100755 --- a/rocketmq-streams-script/pom.xml +++ b/rocketmq-streams-script/pom.xml @@ -28,6 +28,17 @@ ROCKETMQ STREAMS :: script jar + + + org.apache.rocketmq + rocketmq-streams-state + + + + net.agkn + hll + 1.6.0 + org.apache.rocketmq rocketmq-streams-commons diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountAccumulator.java index 2de0e634..b4eae602 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountAccumulator.java @@ -17,12 +17,18 @@ package org.apache.rocketmq.streams.script.function.aggregation; import java.util.Iterator; +import java.util.Map; import java.util.Set; import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.script.annotation.Function; import org.apache.rocketmq.streams.script.annotation.UDAFFunction; import org.apache.rocketmq.streams.script.service.IAccumulator; +import org.apache.rocketmq.streams.state.kv.rocksdb.RocksdbState; +/** + * @author arthur.liang + */ @Function @UDAFFunction("count") public class CountAccumulator implements IAccumulator { @@ -48,10 +54,22 @@ public void accumulate(CountAccum accumulator, Object... parameters) { if (CollectionUtil.isEmpty(parameters) || parameters[0] == null) { return; } - if (parameters[0] instanceof Set) { - //count(distinct(xx)) - //FIXME a trick! use CountDistinctAccumulator instead of the following code - accumulator.count = ((Set)parameters[0]).size(); + if (parameters[0] instanceof DistinctAccumulator2.DistinctAccum2) { + DistinctAccumulator2.DistinctAccum2 distinctAccum2 = (DistinctAccumulator2.DistinctAccum2) parameters[0]; + String prefix = MapKeyUtil.createKey(DistinctAccumulator2.DISTINCT_STATE_PREFIX, distinctAccum2.windowInstanceId, distinctAccum2.groupByMd5); + RocksdbState state = new RocksdbState(); + Iterator> iterator = state.entryIterator(prefix); + int sum = 0; + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + if (entry == null) { + break; + } + sum += 1; + } + accumulator.count = sum; + } else if (parameters[0] instanceof Set) { + accumulator.count = ((Set) parameters[0]).size(); } else { accumulator.count += 1; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountDistinctAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountDistinctAccumulator.java new file mode 100644 index 00000000..1f2ee24d --- /dev/null +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/CountDistinctAccumulator.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.script.function.aggregation; + +import com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; +import java.util.Iterator; +import net.agkn.hll.HLL; +import org.apache.rocketmq.streams.script.annotation.Function; +import org.apache.rocketmq.streams.script.annotation.UDAFFunction; +import org.apache.rocketmq.streams.script.service.IAccumulator; + +/** + * count(distinct) implementation based on hyper-log-log algorithm + * + * @author arthur.liang + */ +@Function +@UDAFFunction("count_distinct") +public class CountDistinctAccumulator implements IAccumulator { + + private static final int SEED = 20210926; + + private static final HashFunction HASH_PRODUCER = Hashing.murmur3_128(SEED); + + @Override public CountDistinctAccum createAccumulator() { + return new CountDistinctAccum(); + } + + @Override public Long getValue(CountDistinctAccum accumulator) { + return accumulator.hll.cardinality(); + } + + @Override public void accumulate(CountDistinctAccum accumulator, Object... parameters) { + if (parameters != null && parameters[0] != null) { + Long hashKey = HASH_PRODUCER.newHasher().putUnencodedChars(parameters[0].toString()).hash().asLong(); + accumulator.hll.addRaw(hashKey); + } + } + + @Override public void merge(CountDistinctAccum accumulator, Iterable its) { + Iterator dvIterator = its.iterator(); + while (dvIterator.hasNext()) { + CountDistinctAccum dvAccum = dvIterator.next(); + if (dvAccum != null && dvAccum.hll != null) { + accumulator.hll.union(dvAccum.hll); + } + } + } + + @Override public void retract(CountDistinctAccum accumulator, String... parameters) { + + } + + public static class CountDistinctAccum { + private final HLL hll = new HLL(30, 8); + } +} diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/DistinctAccumulator.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/DistinctAccumulator.java index fe2c4c04..e7f8e11a 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/DistinctAccumulator.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/DistinctAccumulator.java @@ -25,6 +25,11 @@ import org.apache.rocketmq.streams.script.annotation.UDAFFunction; import org.apache.rocketmq.streams.script.service.IAccumulator; +/** + * distinct operator based memory + * + * @author arthur.liang + */ @Function @UDAFFunction("distinct") public class DistinctAccumulator implements IAccumulator { diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/DistinctAccumulator2.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/DistinctAccumulator2.java new file mode 100644 index 00000000..a4ceecd3 --- /dev/null +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/aggregation/DistinctAccumulator2.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.script.function.aggregation; + +import java.util.Iterator; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.script.annotation.Function; +import org.apache.rocketmq.streams.script.annotation.UDAFFunction; +import org.apache.rocketmq.streams.script.service.IAccumulator; +import org.apache.rocketmq.streams.state.kv.rocksdb.RocksdbState; + +/** + * distinct operator based rocksdb state + * + * @author arthur.liang + */ +@Function +@UDAFFunction("distinct2") +public class DistinctAccumulator2 implements IAccumulator { + + public static final String DISTINCT_STATE_PREFIX = "__distinct__"; + private static final Integer PARAMETER_SIZE = 3; + private static RocksdbState state = new RocksdbState(); + + public static class DistinctAccum2 { + public String windowInstanceId; + public String groupByMd5; + } + + @Override + public DistinctAccum2 createAccumulator() { + return new DistinctAccum2(); + } + + @Override + public DistinctAccum2 getValue(DistinctAccum2 accumulator) { + return accumulator; + } + + @Override + public void accumulate(DistinctAccum2 accumulator, Object... parameters) { + if (CollectionUtil.isEmpty(parameters) || parameters.length != PARAMETER_SIZE) { + return; + } + try { + String value = (String) parameters[0]; + String valueMd5 = StringUtil.createMD5Str(value); + String windowInstanceId = (String) parameters[1]; + if (accumulator.windowInstanceId == null && windowInstanceId != null) { + accumulator.windowInstanceId = windowInstanceId; + } + assert accumulator.windowInstanceId.equalsIgnoreCase(windowInstanceId); + String groupByValue = (String) parameters[2]; + String groupByMd5 = StringUtil.createMD5Str(groupByValue); + if (accumulator.groupByMd5 == null && groupByMd5 != null) { + accumulator.groupByMd5 = groupByMd5; + } + assert accumulator.groupByMd5.equalsIgnoreCase(groupByMd5); + String storeKey = MapKeyUtil.createKey(DISTINCT_STATE_PREFIX, accumulator.windowInstanceId, accumulator.groupByMd5, valueMd5); + state.putIfAbsent(storeKey, value); + } catch (Exception e) { + e.printStackTrace(); + } + } + + @Override + public void merge(DistinctAccum2 accumulator, Iterable its) { + Iterator it = its.iterator(); + while (it.hasNext()) { + DistinctAccum2 commonAccumulator = it.next(); + if (commonAccumulator != null) { + if (accumulator.windowInstanceId == null || accumulator.groupByMd5 == null) { + accumulator.windowInstanceId = commonAccumulator.windowInstanceId; + accumulator.groupByMd5 = commonAccumulator.groupByMd5; + } + assert accumulator.windowInstanceId.equalsIgnoreCase(commonAccumulator.windowInstanceId); + assert accumulator.groupByMd5.equalsIgnoreCase(commonAccumulator.groupByMd5); + } + } + } + + @Override + public void retract(DistinctAccum2 accumulator, String... parameters) { + } + +} \ No newline at end of file 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 3d50f018..1bb2ea0f 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 @@ -79,6 +79,7 @@ public String substringindex(IMessage message, FunctionContext context, return result; } + @FunctionMethod(value = "substr", alias = "substring", comment = "截取从指定的索引startIndex开始扩展到索引endIndex处的字符") public String substringindex(IMessage message, FunctionContext context, @FunctionParamter(comment = "带拆分的字符串代表字列名称或常量值", value = "string") String oriMsg, @@ -94,12 +95,12 @@ public String substringindex(IMessage message, FunctionContext context, return oriMsg.substring(startIndex, endIndex); } - @FunctionMethod(value = "substr", alias = "substring", comment = "截取从指定的索引startIndex处开始扩展到此字符串的结尾") + @FunctionMethod(value = "substr", alias = "blink_substring", comment = "截取从指定的索引startIndex处开始扩展到此字符串的结尾") public String substringindex(IMessage message, FunctionContext context, @FunctionParamter(comment = "带拆分的字符串代表字列名称或常量值", value = "string") String oriMsg, @FunctionParamter(comment = "指定用于拆分原始字段的字符代表列名称或常量值", value = "string") Integer startIndex) { oriMsg = FunctionUtils.getValueString(message, context, oriMsg); - return oriMsg.substring(startIndex); + return oriMsg == null ? null : startIndex >= oriMsg.length() ? "" : oriMsg.substring(startIndex); } @FunctionMethod(value = "blink_substr", alias = "blink_substring", comment = "截取从指定的索引startIndex开始,长度为len的字符,index从1开始,需要做下处理") diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/DefaultFunctionServiceImpl.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/DefaultFunctionServiceImpl.java index e237a87f..88a833bb 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/DefaultFunctionServiceImpl.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/function/service/impl/DefaultFunctionServiceImpl.java @@ -135,6 +135,7 @@ public void registeInterface(String name, Object interfaceObject) { return; } className2InnerInterface.put(name, interfaceObject); + className2InnerInterface.put(name.toUpperCase(), interfaceObject); } /** @@ -151,6 +152,7 @@ public void registeFunction(Object bean) { functionType = FunctionType.UDAF; String name = ((UDAFFunction)udaf).value(); this.className2InnerInterface.put(name, bean); + this.className2InnerInterface.put(name.toUpperCase(), bean); return; } diff --git a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java index 35b173c2..d7b6553d 100644 --- a/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java +++ b/rocketmq-streams-script/src/main/java/org/apache/rocketmq/streams/script/operator/impl/AggregationScript.java @@ -31,7 +31,9 @@ import org.apache.rocketmq.streams.script.function.aggregation.ConcatAccumulator; import org.apache.rocketmq.streams.script.function.aggregation.ConcatDistinctAccumulator; import org.apache.rocketmq.streams.script.function.aggregation.CountAccumulator; +import org.apache.rocketmq.streams.script.function.aggregation.CountDistinctAccumulator; import org.apache.rocketmq.streams.script.function.aggregation.DistinctAccumulator; +import org.apache.rocketmq.streams.script.function.aggregation.DistinctAccumulator2; import org.apache.rocketmq.streams.script.function.aggregation.MaxAccumulator; import org.apache.rocketmq.streams.script.function.aggregation.MinAccumulator; import org.apache.rocketmq.streams.script.function.aggregation.SumAccumulator; @@ -49,10 +51,12 @@ public class AggregationScript implements IStreamOperatorjar ROCKETMQ STREAMS :: state - + + org.apache.rocketmq + rocketmq-streams-commons + + + + org.rocksdb + rocksdbjni + diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/LruState.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/LruState.java new file mode 100644 index 00000000..7ccb149a --- /dev/null +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/LruState.java @@ -0,0 +1,264 @@ +/* + * 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.state; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; + +/** + * lru && o(1) + * + * @author arthur.liang + */ +public final class LruState { + + private int max_size; + + private final T FLAG_VALUE; + + private ConcurrentHashMap> elementMap; + + private LinkedList elementList; + + public LruState(int the_size, T flag) { + max_size = the_size; + FLAG_VALUE = flag; + elementMap = new ConcurrentHashMap<>(max_size); + elementList = new LinkedList<>(FLAG_VALUE); + } + + /** + * add + * + * @param value + * @return + */ + public synchronized boolean add(T value) { + if (FLAG_VALUE.equals(value)) { + return false; + } + Element newElement = new Element(value); + if (elementMap.containsKey(value)) { + Element oldElement = elementMap.get(value); + elementList.median = oldElement.next; + elementList.removeElement(oldElement); + newElement.increase(oldElement.getCounter()); + } else if (elementMap.size() >= max_size) { + Element deletedElement = elementList.removeElement(); + elementMap.remove(deletedElement.getValue()); + } + elementList.addElement(newElement); + elementMap.put(value, newElement); + return true; + } + + /** + * remove + * + * @param value + * @return + */ + public synchronized boolean remove(T value) { + if (!elementMap.containsKey(value)) { + return false; + } + Element theElement = elementMap.get(value); + elementList.removeElement(theElement); + elementMap.remove(value); + return true; + } + + /** + * search + * + * @param value + * @return if contains return the hit counts else return 0 + */ + public synchronized int search(T value) { + if (elementMap.containsKey(value)) { + return elementMap.get(value).getCounter(); + } else { + return 0; + } + } + + /** + * iterator + * + * @return + */ + public synchronized List getAll() { + ArrayList iteratorList = new ArrayList(); + Element pointElement = elementList.head.next; + while (pointElement.next != null) { + iteratorList.add(pointElement.getValue()); + pointElement = pointElement.next; + } + return iteratorList; + } + + public synchronized Iterator iterator() { + return new LruIterator(); + } + + public class LruIterator implements Iterator { + + private Element current = elementList.head.next; + + @Override public boolean hasNext() { + return current != null && !current.equals(elementList.tail); + } + + @Override public T next() { + T value = current.getValue(); + current = current.next; + return value; + } + } + + /** + * counter + * + * @return + */ + public synchronized int count() { + return elementMap.size(); + } + +} + +class Element { + + private T value; + + private volatile int counter; + + Element next; + + Element pre; + + public Element(T element) { + this.value = element; + counter = 1; + } + + public T getValue() { + return this.value; + } + + public void increase(int number) { + counter += number; + } + + public int getCounter() { + return counter; + } + +} + +class LinkedList { + + volatile int current_size = 0; + + Element head; + + Element tail; + + Element median; + + public LinkedList(T flag) { + head = new Element(flag); + tail = new Element(flag); + head.pre = null; + tail.next = null; + } + + /** + * add one element + * + * @param theOne + * @return isSuccess + */ + public boolean addElement(Element theOne) { + if (theOne == null) { + return false; + } + if (current_size == 0) { + head.next = theOne; + theOne.pre = head; + theOne.next = tail; + tail.pre = theOne; + } else { + Element current; + if (median != null && theOne.getCounter() > median.getCounter()) { + current = median.pre; + } else { + current = tail.pre; + } + while (current != null && current != head) { + if (theOne.getCounter() <= current.getCounter()) { + break; + } else { + current = current.pre; + } + } + current.next.pre = theOne; + theOne.next = current.next; + current.next = theOne; + theOne.pre = current; + } + current_size++; + median = theOne; + return true; + } + + /** + * remove the element at the end of list + * + * @return isSuccess + */ + public Element removeElement() { + Element theElement = tail.pre; + if (theElement == null || theElement == head) { + return null; + } + removeElement(theElement); + return theElement; + } + + /** + * remove the special element + * + * @param oneElement + * @return + */ + public boolean removeElement(Element oneElement) { + if (current_size == 0) { + return false; + } + oneElement.next.pre = oneElement.pre; + oneElement.pre.next = oneElement.next; + if (median != null && median.equals(oneElement)) { + median = null; + } + oneElement.next = oneElement.pre = null; + current_size--; + return true; + } +} \ No newline at end of file diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/IKvState.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/IKvState.java new file mode 100644 index 00000000..9450e1c4 --- /dev/null +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/IKvState.java @@ -0,0 +1,119 @@ +/* + * 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.state.kv; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * @param + * @param + * @author arthur.liang + */ +public interface IKvState { + + /** + * Returns the value to which the specified key is mapped, or {@code null} if this map contains no mapping for the + * key. + * + * @param key + * @return value + */ + V get(K key); + + /** + * Returns all values to which all specified keys is mapped + * + * @param key + * @return + */ + Map getAll(List key); + + /** + * Associates the specified value with the specified key in this map (optional operation). If the map previously + * contained a mapping for the key, the old value is replaced by the specified value. + * + * @param key + * @param value + * @return + */ + V put(K key, V value); + + /** + * Associates the specified value with the specified key in this map (optional operation). If the map previously + * contained a mapping for the key, the old value will not be replaced by the specified value. + * + * @param key + * @param value + * @return + */ + V putIfAbsent(K key, V value); + + /** + * Removes the mapping for a key from this map if it is present (optional operation). + * + * @param key + * @return + */ + V remove(K key); + + /** + * Removes the mapping for all keys from this map if it is present (optional operation). + * + * @param keys + */ + void removeAll(List keys); + + // Bulk Operations + + /** + * Copies all of the mappings from the specified map to this map (optional operation). + * + * @param map + */ + void putAll(Map map); + + /** + * Removes all of the mappings from this map (optional operation). The map will be empty after this call returns. + */ + void clear(); + + /** + * Returns a {@link Set} view of the keys contained in this map. + * + * @return a set view of the keys contained in this map + */ + Iterator keyIterator(); + + /** + * Returns a {@link Set} view of the mappings contained in this map. + * + * @return a set view of the mappings contained in this map + */ + Iterator> entryIterator(); + + /** + * Returns a {@link Set} view of the specified prefix mappings contained in this map. + * + * @param prefix + * @returna set view of the mappings contained in this map + */ + Iterator> entryIterator(String prefix); + +} diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksDBOperator.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksDBOperator.java new file mode 100644 index 00000000..c121eac5 --- /dev/null +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksDBOperator.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.state.kv.rocksdb; + +import java.io.File; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.rocketmq.streams.common.utils.FileUtil; +import org.apache.rocketmq.streams.common.utils.RuntimeUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; +import org.rocksdb.TtlDB; +import org.rocksdb.WriteOptions; + +public class RocksDBOperator { + + protected static String DB_PATH = "/tmp/rocksdb"; + + protected static String UTF8 = "UTF8"; + + protected static AtomicBoolean hasCreate = new AtomicBoolean(false); + + protected static RocksDB rocksDB; + + protected WriteOptions writeOptions = new WriteOptions(); + + static { + RocksDB.loadLibrary(); + } + + public RocksDBOperator() { + this(FileUtil.concatFilePath(StringUtil.isEmpty(FileUtil.getJarPath()) ? DB_PATH + File.separator + RuntimeUtil.getDipperInstanceId() : FileUtil.getJarPath() + File.separator + RuntimeUtil.getDipperInstanceId(), "rocksdb")); + } + + public RocksDBOperator(String rocksdbFilePath) { + if (hasCreate.compareAndSet(false, true)) { + synchronized (RocksDBOperator.class) { + if (RocksDBOperator.rocksDB == null) { + synchronized (RocksDBOperator.class) { + if (RocksDBOperator.rocksDB == null) { + try (final Options options = new Options().setCreateIfMissing(true)) { + + try { + File dir = new File(rocksdbFilePath); + if (dir.exists()) { + dir.delete(); + } + dir.mkdirs(); + final TtlDB db = TtlDB.open(options, rocksdbFilePath, 10800, false); + RocksDBOperator.rocksDB = db; + writeOptions.setSync(true); + } catch (RocksDBException e) { + throw new RuntimeException("create rocksdb error " + e.getMessage()); + } + } + } + } + } + } + } + } + + public RocksDB getInstance() { + if (rocksDB == null) { + synchronized (RocksDBOperator.class) { + if (rocksDB == null) { + RocksDBOperator operator = new RocksDBOperator(); + if (rocksDB != null) { + return rocksDB; + } else { + throw new RuntimeException("failed in creating rocksdb!"); + } + } + } + } + return rocksDB; + } + +} diff --git a/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksdbState.java b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksdbState.java new file mode 100644 index 00000000..3e0f38b5 --- /dev/null +++ b/rocketmq-streams-state/src/main/java/org/apache/rocketmq/streams/state/kv/rocksdb/RocksdbState.java @@ -0,0 +1,264 @@ +/* + * 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.state.kv.rocksdb; + +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.state.LruState; +import org.apache.rocketmq.streams.state.kv.IKvState; +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksDBException; +import org.rocksdb.RocksIterator; +import org.rocksdb.WriteBatch; +import org.rocksdb.WriteOptions; + +import static org.apache.rocketmq.streams.state.kv.rocksdb.RocksDBOperator.UTF8; + +/** + * kv state based rocksdb + * + * @author arthur.liang + */ +public class RocksdbState implements IKvState { + + private static RocksDBOperator operator = new RocksDBOperator(); + + private final LruState cache = new LruState<>(100, ""); + + private final static Byte SIGN = 1; + + @Override public String get(String key) { + try { + return getValueFromByte(operator.getInstance().get(getKeyBytes(key))); + } catch (Exception e) { + return null; + } + } + + @Override public Map getAll(List keys) { + if (CollectionUtil.isEmpty(keys)) { + return new HashMap<>(4); + } + List keyByteList = new ArrayList<>(); + List keyStrList = new ArrayList<>(); + for (String key : keys) { + keyByteList.add(getKeyBytes(key)); + keyStrList.add(key); + } + try { + Map resultMap = new HashMap<>(keys.size()); + Map map = operator.getInstance().multiGet(keyByteList); + Iterator> it = map.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + String key = getValueFromByte(entry.getKey()); + String value = getValueFromByte(entry.getValue()); + resultMap.put(key, value); + } + return resultMap; + } catch (RocksDBException e) { + throw new RuntimeException("failed in getting all from rocksdb!", e); + } + } + + @Override public String put(String key, String value) { + Map map = new HashMap(4) {{ + put(key, value); + }}; + putAll(map); + return null; + } + + @Override public String putIfAbsent(String key, String value) { + if (cache.search(key) > 0) { + return null; + } + put(key, value); + cache.add(key); + return null; + } + + @Override public void putAll(Map map) { + if (map == null) { + return; + } + try { + WriteBatch writeBatch = new WriteBatch(); + Iterator> it = map.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + String key = entry.getKey(); + String value = entry.getValue(); + writeBatch.put(key.getBytes(UTF8), value.getBytes(UTF8)); + } + + WriteOptions writeOptions = new WriteOptions(); + writeOptions.setSync(false); + writeOptions.setDisableWAL(true); + operator.getInstance().write(writeOptions, writeBatch); + writeBatch.close(); + writeOptions.close(); + } catch (Exception e) { + throw new RuntimeException("failed in putting all into rocksdb!", e); + } + } + + @Override public String remove(String key) { + try { + operator.getInstance().delete(getKeyBytes(key)); + } catch (RocksDBException e) { + throw new RuntimeException("failed in removing all from rocksdb! " + key, e); + } + return null; + } + + @Override public void removeAll(List keys) { + for (String key : keys) { + try { + operator.getInstance().delete(getKeyBytes(key)); + } catch (RocksDBException e) { + throw new RuntimeException("failed in removing all from rocksdb! " + key, e); + } + } + } + + @Override public void clear() { + } + + @Override public Iterator keyIterator() { + return null; + } + + @Override public Iterator> entryIterator() { + return null; + } + + @Override public Iterator> entryIterator(String prefix) { + return new RocksDBIterator(prefix); + } + + /** + * 把key转化成byte + * + * @param key + * @return + */ + protected byte[] getKeyBytes(String key) { + try { + if (StringUtil.isEmpty(key)) { + return null; + } + return key.getBytes(UTF8); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("failed in getting byte[] from key! " + key, e); + } + } + + /** + * 把byte转化成值 + * + * @param bytes + * @return + */ + protected static String getValueFromByte(byte[] bytes) { + try { + return new String(bytes, UTF8); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + public static class RocksDBIterator implements Iterator> { + + protected AtomicBoolean hasInit = new AtomicBoolean(false); + + private ReadOptions readOptions = new ReadOptions(); + + private RocksIterator iter; + + protected String keyPrefix; + + public RocksDBIterator(String keyPrefix) { + readOptions.setPrefixSameAsStart(true).setTotalOrderSeek(true); + iter = operator.getInstance().newIterator(readOptions); + this.keyPrefix = keyPrefix; + } + + @Override public boolean hasNext() { + if (hasInit.compareAndSet(false, true)) { + iter.seek(keyPrefix.getBytes()); + } + return iter.isValid(); + } + + @Override public Map.Entry next() { + String key = new String(iter.key()); + if (!key.startsWith(keyPrefix)) { + return null; + } + String value = getValueFromByte(iter.value()); + iter.next(); + return new Element(key, value); + } + + } + + private static class Element implements Map.Entry { + + private Pair pair; + + private Element() { + + } + + public Element(String key, String value) { + pair = Pair.of(key, value); + } + + @Override public String getKey() { + if (pair != null) { + return pair.getKey(); + } + return null; + } + + @Override public String getValue() { + if (pair != null) { + return pair.getRight(); + } + return null; + } + + @Override public String setValue(String value) { + if (pair != null) { + String old = pair.getRight(); + pair.setValue(value); + return old; + } + return null; + } + } + +} diff --git a/rocketmq-streams-state/src/test/java/org/apache/rocketmq/streams/state/kv/TestLruState.java b/rocketmq-streams-state/src/test/java/org/apache/rocketmq/streams/state/kv/TestLruState.java new file mode 100644 index 00000000..b7e43ae3 --- /dev/null +++ b/rocketmq-streams-state/src/test/java/org/apache/rocketmq/streams/state/kv/TestLruState.java @@ -0,0 +1,62 @@ +package org.apache.rocketmq.streams.state.kv; + +import java.util.Iterator; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import org.apache.rocketmq.streams.state.LruState; +import org.junit.Assert; +import org.junit.Test; + +public class TestLruState { + + @Test + public void testBasic() { + String content = "lru"; + int size = 100; + final LruState lruState = new LruState<>(10, ""); + // + for (int i = 0; i < size; i++) { + lruState.add(content); + } + Assert.assertEquals(size, lruState.search(content)); + Assert.assertEquals(1, lruState.count()); + lruState.remove(content); + Assert.assertEquals(0, lruState.search(content)); + Assert.assertEquals(0, lruState.count()); + // + for (int i = 0; i < 11; i++) { + String value = content + "_" + i; + for (int j = 0; j < i + 1; j++) { + lruState.add(value); + } + } + Iterator iterator = lruState.iterator(); + int count = 0; + String theValue = null; + while (iterator.hasNext()) { + count++; + theValue = iterator.next(); + Assert.assertNotEquals(content + "_0", theValue); + } + Assert.assertEquals(content + "_1", theValue); + Assert.assertEquals(10, count); + } + + @Test + public void testConcurrent() { + String content = "lru"; + final LruState lruState = new LruState<>(10, ""); + ExecutorService poolService = Executors.newFixedThreadPool(10); + final Random random = new Random(System.currentTimeMillis()); + for (int i = 0; i < 1000; i++) { + final int index = i; + poolService.execute(() -> { + lruState.add(content + random.nextInt(11)); + }); + } + Assert.assertEquals(10, lruState.count()); + } +} + + diff --git a/rocketmq-streams-state/src/test/java/org/apache/rocketmq/streams/state/kv/TestRocksdbState.java b/rocketmq-streams-state/src/test/java/org/apache/rocketmq/streams/state/kv/TestRocksdbState.java new file mode 100644 index 00000000..cf1a8b7b --- /dev/null +++ b/rocketmq-streams-state/src/test/java/org/apache/rocketmq/streams/state/kv/TestRocksdbState.java @@ -0,0 +1,91 @@ +package org.apache.rocketmq.streams.state.kv; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.state.kv.rocksdb.RocksdbState; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +/** + * @author arthur.liang + */ +public class TestRocksdbState { + + private RocksdbState rocksdbState = new RocksdbState(); + + private String key = "window_id"; + + private String value = "window_value"; + + private int sampleSize = 8; + + @Before + public void testAddAll() { + rocksdbState.put(key, value); + Map stateMap = new HashMap<>(8); + for (int i = 0; i < sampleSize; i++) { + stateMap.put(key + "_" + i, value + "_" + i); + } + rocksdbState.putAll(stateMap); + } + + @Test + public void testGetAll() { + String singleValue = rocksdbState.get(key); + Assert.assertEquals(value, singleValue); + List keys = new ArrayList(sampleSize); + for (int i = 0; i < sampleSize; i++) { + keys.add(key + "_" + i); + } + Map valueMap = rocksdbState.getAll(keys); + Assert.assertEquals(8, valueMap.size()); + singleValue = rocksdbState.get("any_key"); + Assert.assertEquals(null, singleValue); + } + + @Test + public void testIterator() { + Iterator> iterator = rocksdbState.entryIterator(key); + Map valueMap = new HashMap<>(sampleSize); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + if (entry == null) { + break; + } + valueMap.put(entry.getKey(), entry.getValue()); + } + Assert.assertEquals(9, valueMap.size()); + } + + @Test + public void testDelete() { + rocksdbState.remove(key); + String singleValue = rocksdbState.get(key); + Assert.assertEquals(null, singleValue); + List keys = new ArrayList<>(sampleSize); + for (int i = 0; i < sampleSize; i++) { + keys.add(key + "_" + i); + } + rocksdbState.removeAll(keys); + Map valueMap = rocksdbState.getAll(keys); + Assert.assertEquals(0, valueMap.size()); + } + + @Test + public void testOverWrite() { + String replaceValue = value + "_new"; + rocksdbState.put(key, replaceValue); + String replaceResult = rocksdbState.get(key); + Assert.assertEquals(replaceResult, replaceValue); + } + + @Test + public void testNotOverWrite() { + + } + +} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java index a044b174..a72bbc76 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java @@ -18,9 +18,11 @@ import org.apache.rocketmq.streams.common.component.ComponentCreator; import org.apache.rocketmq.streams.common.configure.ConfigureFileKey; +import org.apache.rocketmq.streams.common.topology.model.IWindow; import org.apache.rocketmq.streams.common.utils.StringUtil; import org.apache.rocketmq.streams.window.operator.AbstractWindow; import org.apache.rocketmq.streams.window.operator.impl.OverWindow; +import org.apache.rocketmq.streams.window.operator.impl.SessionOperator; import org.apache.rocketmq.streams.window.operator.impl.WindowOperator; import org.apache.rocketmq.streams.window.operator.join.JoinWindow; @@ -31,11 +33,14 @@ public class WindowBuilder { public static final int DEFAULT_WINDOW_INTERVAL_SIZE_MINUTE = 5; private static boolean TEST_MODE = false; - public static WindowOperator createWindow() { - if (!TEST_MODE) { - return new WindowOperator(); - } else { - return new WindowOperator(); + public static WindowOperator createWindow(String type) { + switch (type){ + case IWindow.TUMBLE_WINDOW: + case IWindow.HOP_WINDOW: + default: + return new WindowOperator(); + case IWindow.SESSION_WINDOW: + return new SessionOperator(); } } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/EventTimeManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/EventTimeManager.java index 8850ada7..990b6902 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/EventTimeManager.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/EventTimeManager.java @@ -19,14 +19,19 @@ import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.commons.lang3.tuple.Pair; import org.apache.rocketmq.streams.common.channel.source.ISource; import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.topology.model.IWindow; import org.apache.rocketmq.streams.window.operator.AbstractWindow; public class EventTimeManager { private Map eventTimeManagerMap=new HashMap<>(); protected ISource source; + private Map> eventTimeIncreasementMap = new ConcurrentHashMap<>(); + public void updateEventTime(IMessage message, AbstractWindow window){ String queueId=message.getHeader().getQueueId(); @@ -43,13 +48,25 @@ public void updateEventTime(IMessage message, AbstractWindow window){ splitEventTimeManager.updateEventTime(message,window); } - - - public Long getMaxEventTime(String queueId){ - - SplitEventTimeManager splitEventTimeManager=eventTimeManagerMap.get(queueId); - if(splitEventTimeManager!=null){ - return splitEventTimeManager.getMaxEventTime(); + public Long getMaxEventTime(String queueId) { + SplitEventTimeManager splitEventTimeManager = eventTimeManagerMap.get(queueId); + if (splitEventTimeManager != null) { + Long currentMaxEventTime = splitEventTimeManager.getMaxEventTime(); + if (eventTimeIncreasementMap.containsKey(queueId)) { + Long lastMaxEventTime = eventTimeIncreasementMap.get(queueId).getKey(); + if (lastMaxEventTime.equals(currentMaxEventTime)) { + //increase event time as time flies to solve batch data processing issue + if (System.currentTimeMillis() - eventTimeIncreasementMap.get(queueId).getRight() > IWindow.SYS_DELAY_TIME) { + Long newEventTime = lastMaxEventTime + (System.currentTimeMillis() - eventTimeIncreasementMap.get(queueId).getRight()); + return newEventTime; + } + } else { + eventTimeIncreasementMap.put(queueId, Pair.of(currentMaxEventTime, System.currentTimeMillis())); + } + } else { + eventTimeIncreasementMap.put(queueId, Pair.of(currentMaxEventTime, System.currentTimeMillis())); + } + return eventTimeIncreasementMap.get(queueId).getLeft(); } return null; } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/SplitEventTimeManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/SplitEventTimeManager.java index 70742f3a..9e5faf3e 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/SplitEventTimeManager.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/fire/SplitEventTimeManager.java @@ -53,6 +53,8 @@ public class SplitEventTimeManager { protected volatile boolean isAllSplitReceived=false; protected transient String queueId; + private static Long splitReadyTime; + public SplitEventTimeManager(ISource source,String queueId){ this.source=source; this.queueId=queueId; @@ -155,9 +157,22 @@ protected boolean isSplitsReceiver(){ if(this.splitsGroupByInstance==null){ return false; } - if(workingSplitSize==messageSplitId2MaxTime.size()){ - this.isAllSplitReceived=true; + //add time out policy: no necessary waiting for other split + if (splitReadyTime == null) { + synchronized (this) { + if (splitReadyTime == null) { + splitReadyTime = System.currentTimeMillis(); + } + } + } + if (workingSplitSize == messageSplitId2MaxTime.size()) { + this.isAllSplitReceived = true; return true; + } else { + if (System.currentTimeMillis() - splitReadyTime >= 1000 * 60) { + this.isAllSplitReceived = true; + return true; + } } return false; } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/FireMode.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/FireMode.java new file mode 100644 index 00000000..24abef51 --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/FireMode.java @@ -0,0 +1,45 @@ +package org.apache.rocketmq.streams.window.model; + +/** + * the mode for supporting more scene + * + * @author arthur + */ + +public enum FireMode { + + /** + * 常规触发(最大延迟时间<=窗口大小) + */ + NORMAL(0), + + /** + * 分段触发(最大延迟时间>窗口大小) + */ + PARTITIONED(1), + + /** + * 增量触发(最大延迟时间>窗口大小) + */ + ACCUMULATED(2); + + int value; + + private FireMode(int num) { + this.value = num; + } + + public static FireMode valueOf(int theValue) { + switch (theValue) { + case 0: + return NORMAL; + case 1: + return PARTITIONED; + case 2: + return ACCUMULATED; + default: + return null; + } + } + +} 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 c5759767..6b5090a2 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 @@ -277,7 +277,7 @@ public static List getOrCreateWindowInstance(AbstractWindow wind if(window.getFireMode()==2&&fire.getTime()==end.getTime()&&waterMarkMinute>0){ Date clearWindowInstanceFireTime=DateUtil.addDate(TimeUnit.SECONDS,end, waterMarkMinute * timeUnitAdjust); WindowInstance lastWindowInstance=window.createWindowInstance(DateUtil.format(begin), DateUtil.format(end),DateUtil.format(clearWindowInstanceFireTime) , queueId); - window.getWindowInstanceMap().putIfAbsent(lastWindowInstance.createWindowInstanceTriggerId(),lastWindowInstance); + window.registerWindowInstance(lastWindowInstance); window.getSqlCache().addCache(new SQLElement(queueId,lastWindowInstance.createWindowInstanceId(),ORMUtil.createBatchReplacetSQL(lastWindowInstance))); window.getWindowFireSource().registFireWindowInstanceIfNotExist(lastWindowInstance,window); } @@ -294,7 +294,7 @@ public static List getOrCreateWindowInstance(AbstractWindow wind String endTime = DateUtil.format(end); String fireTime = DateUtil.format(fire); String windowInstanceTriggerId = window.createWindowInstance(startTime, endTime, fireTime, queueId).createWindowInstanceTriggerId(); - WindowInstance windowInstance = window.getWindowInstanceMap().get(windowInstanceTriggerId); + WindowInstance windowInstance = window.searchWindowInstance(windowInstanceTriggerId); if (windowInstance == null) { lostWindowTimeList.add(Pair.of(startTime, endTime)); lostFireList.add(fireTime); @@ -308,41 +308,12 @@ public static List getOrCreateWindowInstance(AbstractWindow wind instanceList.addAll(lostInstanceList); for (WindowInstance windowInstance : instanceList) { - window.getWindowInstanceMap().putIfAbsent(windowInstance.createWindowInstanceTriggerId(), windowInstance); + window.registerWindowInstance(windowInstance); } return instanceList; } - //public WindowInstance copy() { - // WindowInstance windowInstance=new WindowInstance(); - // windowInstance.setNewWindowInstance(this.getNewWindowInstance()); - // windowInstance.setVersion(this.version); - // windowInstance.setStartTime(this.startTime); - // windowInstance.setEndTime(this.endTime); - // windowInstance.setStatus(this.status); - // windowInstance.setWindowNameSpace(this.windowNameSpace); - // windowInstance.setWindowName(this.windowName); - // windowInstance.setFireTime(this.fireTime); - // windowInstance.setWindowInstanceKey(this.windowInstanceKey); - // windowInstance.setGmtCreate(this.gmtCreate); - // windowInstance.setGmtModified(this.gmtModified); - // return windowInstance; - //} - - //public WindowInstance toMd5Instance() { - // WindowInstance instance = copy(); - // instance.setWindowInstanceKey(StringUtil.createMD5Str(instance.getWindowInstanceKey())); - // return instance; - //} - - //public WindowInstance toOriginInstance(boolean supportOutDate) { - // WindowInstance instance = copy(); - // instance.setWindowInstanceKey(null); - // instance.createWindowInstanceId(supportOutDate); - // return instance; - //} - public String getStartTime() { return startTime; } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueProcessor.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueProcessor.java index ba03beb9..f6560f1b 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueProcessor.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueProcessor.java @@ -23,7 +23,9 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.commons.lang3.tuple.MutablePair; +import org.apache.commons.lang3.tuple.Pair; import org.apache.rocketmq.streams.common.context.MessageOffset; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.SQLUtil; @@ -117,7 +119,8 @@ public void deleteSplitNum(WindowInstance instance) { this.windowOffsetMap.remove(key); List dels=new ArrayList<>(); dels.add(windowMaxValue.getMsgKey()); - String sql="delete from "+ORMUtil.getTableName(WindowMaxValue.class)+" where msg_key in("+SQLUtil.createInSql(dels)+")"; + List> likePairList = dels.stream().map(value -> Pair.of("msg_key", value + "%")).collect(Collectors.toList()); + String sql = "delete from " + ORMUtil.getTableName(WindowMaxValue.class) + " where " + SQLUtil.createLikeSql(likePairList); if(sqlCache!=null){ sqlCache.addCache(new SQLElement(this.splitId,instance.createWindowInstanceId(),sql)); @@ -168,7 +171,7 @@ public Map loadOffset(String name) { } String keyPrefix=MapKeyUtil.createKey(name,splitId); - String sql="select * from "+ ORMUtil.getTableName(WindowMaxValue.class)+ " where msg_key like '"+keyPrefix+"%'"; + String sql = "select * from " + ORMUtil.getTableName(WindowMaxValue.class) + " where configure_name like '%" + name + "%' and partition like '%" + splitId + "%'"; List windowMaxValues = ORMUtil.queryForList(sql, null, WindowMaxValue.class); if(windowMaxValues==null||windowMaxValues.size()==0){ return result; diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java index 15277d06..978f1915 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java @@ -26,7 +26,7 @@ import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.shuffle.ShuffleChannel; -import org.apache.rocketmq.streams.window.source.WindowRireSource; +import org.apache.rocketmq.streams.window.source.WindowFireSource; import org.apache.rocketmq.streams.window.storage.WindowStorage; public abstract class AbstractShuffleWindow extends AbstractWindow implements IAfterConfigurableRefreshListener { @@ -44,7 +44,7 @@ protected boolean initConfigurable() { @Override public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) { if (hasCreated.compareAndSet(false, true)) { - this.windowFireSource = new WindowRireSource(this); + this.windowFireSource = new WindowFireSource(this); this.windowFireSource.init(); this.windowFireSource.start(getFireReceiver()); this.shuffleChannel = new ShuffleChannel(this); 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 2cf82456..7328a610 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 @@ -39,6 +39,7 @@ import org.apache.rocketmq.streams.common.topology.stages.udf.IReducer; import org.apache.rocketmq.streams.common.utils.Base64Utils; import org.apache.rocketmq.streams.common.utils.InstantiationUtil; +import org.apache.rocketmq.streams.common.utils.TraceUtil; import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; import org.apache.rocketmq.streams.window.debug.DebugWriter; import org.apache.rocketmq.streams.window.fire.EventTimeManager; @@ -47,7 +48,7 @@ import org.apache.rocketmq.streams.window.model.WindowCache; import org.apache.rocketmq.streams.window.offset.IWindowMaxValueManager; import org.apache.rocketmq.streams.window.offset.WindowMaxValueManager; -import org.apache.rocketmq.streams.window.source.WindowRireSource; +import org.apache.rocketmq.streams.window.source.WindowFireSource; import org.apache.rocketmq.streams.window.state.impl.WindowValue; import org.apache.rocketmq.streams.common.context.AbstractContext; import org.apache.rocketmq.streams.common.context.IMessage; @@ -119,7 +120,7 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo */ protected int slideInterval; /** - * 主要是做兼容,以前设计的窗口时间是分钟为单位,如果有秒作为窗口时间的,通过设置timeUntiAdjust=1来实现。 后续需要调整成直接秒级窗口 + * 主要是做兼容,以前设计的窗口时间是分钟为单位,如果有秒作为窗口时间的,通过设置timeUnitAdjust=1来实现。 后续需要调整成直接秒级窗口 */ protected int timeUnitAdjust=60; /** @@ -166,7 +167,7 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo protected transient Map columnProjectMap = new HashMap<>(); /** - * 当前计算节点的PipeLine里的Window实例对象,方便基于时间快速定位 key:namespace;configName(这里理解成windowName);startTime;endTime value:WindowInstance + * 当前计算节点的PipeLine里的Window实例对象 */ protected transient ConcurrentHashMap windowInstanceMap = new ConcurrentHashMap<>(); @@ -189,7 +190,7 @@ public abstract class AbstractWindow extends BasedConfigurable implements IWindo protected volatile transient WindowCache windowCache; protected transient WindowStorage storage; - protected transient WindowRireSource windowFireSource; + protected transient WindowFireSource windowFireSource; protected transient SQLCache sqlCache; protected transient EventTimeManager eventTimeManager; @@ -269,9 +270,7 @@ public AbstractContext doMessage(IMessage message, AbstractContext con msg.put(AbstractWindow.class.getSimpleName(), this); eventTimeManager.setSource(message.getHeader().getSource()); windowCache.batchAdd(message); - //主要为了在单元测试中,写入和触发一体化使用,无实际意义,不要在业务场景使用这个字段 - - // TraceUtil.debug(message.getHeader().getTraceId(), "origin message in", message.getMessageBody().toJSONString()); + TraceUtil.debug(message.getHeader().getTraceId(), "origin message in"); return context; } @@ -466,6 +465,38 @@ public List queryOrCreateWindowInstance(IMessage message,String queueId); } + public WindowInstance registerWindowInstance(WindowInstance windowInstance) { + return registerWindowInstance(windowInstance.createWindowInstanceTriggerId(), windowInstance); + } + + /** + * register window instance with indexId key + * @param indexId + * @param windowInstance + */ + protected WindowInstance registerWindowInstance(String indexId, WindowInstance windowInstance) { + return windowInstanceMap.putIfAbsent(indexId, windowInstance); + } + + /** + * search window instance by using index id, return null if not exist + * + * @param indexId + * @return + */ + public WindowInstance searchWindowInstance(String indexId) { + return windowInstanceMap.getOrDefault(indexId, null); + } + + /** + * logout window instance by using index id + * + * @param indexId + */ + public void logoutWindowInstance(String indexId) { + windowInstanceMap.remove(indexId); + } + /** * 获取window处理的消息中最大的时间 * @@ -607,11 +638,11 @@ public void setSizeInterval(int sizeInterval) { this.sizeInterval = sizeInterval; } - public ConcurrentHashMap getWindowInstanceMap() { + private ConcurrentHashMap getWindowInstanceMap() { return windowInstanceMap; } - public void setWindowInstanceMap( + private void setWindowInstanceMap( ConcurrentHashMap windowInstanceMap) { this.windowInstanceMap = windowInstanceMap; } @@ -730,7 +761,7 @@ public WindowStorage getStorage() { return storage; } - public WindowRireSource getWindowFireSource() { + public WindowFireSource getWindowFireSource() { return windowFireSource; } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java new file mode 100644 index 00000000..0e5f8fa8 --- /dev/null +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java @@ -0,0 +1,492 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.streams.window.operator.impl; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.rocketmq.streams.common.context.IMessage; +import org.apache.rocketmq.streams.common.utils.Base64Utils; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; +import org.apache.rocketmq.streams.common.utils.DateUtil; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.common.utils.TraceUtil; +import org.apache.rocketmq.streams.window.model.WindowInstance; +import org.apache.rocketmq.streams.window.state.WindowBaseValue; +import org.apache.rocketmq.streams.window.state.impl.WindowValue; +import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator; + +/** + * an implementation of session window to save extra memory for different group by window instances + * + * @author arthur + */ +public class SessionOperator extends WindowOperator { + + + protected static final Log LOG = LogFactory.getLog(SessionOperator.class); + + public static final String SESSION_WINDOW_BEGIN_TIME = "1970-01-01"; + + public static final String SESSION_WINDOW_END_TIME = "9999-01-01"; + + private static final String SESSION_DATETIME_PATTERN = "yyyy-MM-dd HH:mm:ss"; + + private static final String ORDER_BY_FIRE_TIME_PREFIX = "_order_by_fire_time_"; + + /** + * 会话窗口的超时时间,时间单位时秒,默认10分钟 + */ + protected int sessionTimeOut = 10 * 60; + + private transient Object lock = new Object(); + + public SessionOperator() { + + } + + public SessionOperator(Integer timeout) { + this.sessionTimeOut = Optional.ofNullable(timeout).orElse(sessionTimeOut); + } + + public int getSessionTimeOut() { + return sessionTimeOut; + } + + public void setSessionTimeOut(int sessionTimeOut) { + this.sessionTimeOut = sessionTimeOut; + } + + @Override + protected boolean initConfigurable() { + // + return super.initConfigurable(); + } + + /** + * one queue own only one window instance, init the first fire time for the instance + * + * @param message + * @param queueId + * @return + */ + @Override + public List queryOrCreateWindowInstance(IMessage message, String queueId) { + WindowInstance instance = createWindowInstance(SESSION_WINDOW_BEGIN_TIME, SESSION_WINDOW_END_TIME, null, queueId); + String windowInstanceId = instance.createWindowInstanceId(); + WindowInstance existWindowInstance = searchWindowInstance(windowInstanceId); + if (existWindowInstance == null) { + Pair startEndPair = getSessionTime(message); + Date fireDate = DateUtil.addDate(TimeUnit.SECONDS, startEndPair.getRight(), waterMarkMinute * timeUnitAdjust); + //out of order data, normal fire mode considered only + Long maxEventTime = getMaxEventTime(queueId); + if (maxEventTime == null) { + LOG.warn("use current time as max event time!"); + maxEventTime = System.currentTimeMillis(); + } + if (fireDate.getTime() <= maxEventTime) { + LOG.warn("message is discarded as out of date! fire time: " + fireDate.getTime() + " max event time: " + maxEventTime); + return new ArrayList<>(); + } + instance.setFireTime(DateUtil.format(fireDate, SESSION_DATETIME_PATTERN)); + registerWindowInstance(instance); + } + return new ArrayList() {{ + add(searchWindowInstance(windowInstanceId)); + }}; + } + + @Override public WindowInstance registerWindowInstance(WindowInstance windowInstance) { + return super.registerWindowInstance(windowInstance.createWindowInstanceId(), windowInstance); + } + + @Override + public void shuffleCalculate(List messages, WindowInstance instance, String queueId) { + /** + * 1、消息分组:获得分组的groupBy值和对应的消息 + * 2、获取已有所有分组的窗口计算结果:1)通过queueId、instance和groupBy计算存储的key;2)调用存储的获取接口; + * 3、消息聚合计算:对每一条消息,如果存在可以合并的窗口,则修改窗口信息(备注:窗口信息的维护都是在WindowValue而非WindowInstance;窗口信息的改变对应的存储也要进行清除);如果不存在,则创建新的会话窗口 + * 4、窗口的合并计算:新消息导致不同窗口合并成一个大的会话窗口 + * 5、窗口结果存储:剩余的未触发的窗口结果进行本地存储(覆盖和新增)和远程存储(先删除,后增加),并进行窗口的前缀存储(覆盖和新增); + */ + synchronized (lock) { + // + List groupSortedByOffset = new ArrayList<>(); + Map> groupBy = groupByGroupName(messages, groupSortedByOffset); + int groupSize = groupSortedByOffset.size(); + // + Map value2StoreMap = new HashMap<>(groupSize); + for (String groupValue : groupSortedByOffset) { + String storeKey = createStoreKey(queueId, groupValue, instance); + value2StoreMap.put(groupValue, storeKey); + } + Map> storeValueMap = storage.multiGetList(WindowValue.class, new ArrayList<>(value2StoreMap.values())); + // + Iterator>> iterator = groupBy.entrySet().iterator(); + Map> resultMap = new HashMap<>(groupSize); + while (iterator.hasNext()) { + Map.Entry> entry = iterator.next(); + String groupValue = entry.getKey(); + String storeKey = value2StoreMap.get(groupValue); + List groupMessageList = entry.getValue(); + Map id2ValueMap = new HashMap<>(groupMessageList.size()); + List valueList = storeValueMap.getOrDefault(storeKey, new ArrayList<>()); + for (WindowValue value : valueList) { + id2ValueMap.put(value.getPartitionNum(), value); + } + for (IMessage message : groupMessageList) { + // + WindowValue windowValue = queryOrCreateWindowValue(instance, queueId, groupValue, message, valueList, storeKey); + windowValue.calculate(this, message); + //region trace + String traceId = message.getMessageBody().getString("SHUFFLE_TRACE_ID"); + if (!StringUtil.isEmpty(traceId)) { + try { + String result = new String(Base64Utils.decode(windowValue.getComputedColumnResult()), "UTF-8"); + TraceUtil.debug(traceId, "shuffle message out " + groupValue, String.valueOf(windowValue.getPartitionNum()), windowValue.getStartTime(), windowValue.getEndTime(), result); + } catch (Exception e) { + + } + } + //endregion + id2ValueMap.put(windowValue.getPartitionNum(), windowValue); + } + //merge values + List groupValueList = mergeWindowValue(new ArrayList<>(id2ValueMap.values()), instance, queueId); + resultMap.put(storeKey, groupValueList); + } + // + store(resultMap, instance, queueId); + } + + } + + /** + * query window value, merge value at the same time: 1) merge message into the value; 2) merge different values + * + * @param windowInstance + * @param queueId + * @param groupByValue + * @param message + * @param valueList + * @param storeKey + * @return + */ + private WindowValue queryOrCreateWindowValue(WindowInstance windowInstance, String queueId, String groupByValue, + IMessage message, List valueList, String storeKey) { + // + if (CollectionUtil.isEmpty(valueList)) { + return createWindowValue(queueId, groupByValue, windowInstance, message, storeKey); + } + //put keys to be deleted here and delete them at last + List deletePrefixKeyList = new ArrayList<>(); + // + for (WindowValue value : valueList) { + Date sessionBegin = DateUtil.parseTime(value.getStartTime()); + Date sessionEnd = DateUtil.parseTime(value.getEndTime()); + Pair startEndPair = getSessionTime(message); + Date messageBegin = startEndPair.getLeft(); + Date messageEnd = startEndPair.getRight(); + if (messageBegin.compareTo(sessionBegin) >= 0 && messageBegin.compareTo(sessionEnd) < 0) { + sessionEnd = messageEnd; + Date sessionFire = DateUtil.addDate(TimeUnit.SECONDS, sessionEnd, waterMarkMinute * timeUnitAdjust); + value.setEndTime(DateUtil.format(sessionEnd, SESSION_DATETIME_PATTERN)); + //clean order storage as sort field 'fireTime' changed + String existPartitionNumKey = createPrefixKey(value, windowInstance, queueId); + deletePrefixKeyList.add(existPartitionNumKey); + deletePrefixValue(deletePrefixKeyList); + // + value.setFireTime(DateUtil.format(sessionFire, SESSION_DATETIME_PATTERN)); + return value; + } else if (messageBegin.compareTo(sessionBegin) < 0 && messageEnd.compareTo(sessionBegin) > 0) { + sessionBegin = messageBegin; + value.setStartTime(DateUtil.format(sessionBegin, SESSION_DATETIME_PATTERN)); + return value; + } + } + // + WindowValue newValue = createWindowValue(queueId, groupByValue, windowInstance, message, storeKey); + valueList.add(newValue); + return newValue; + } + + private List mergeWindowValue(List allValueList, WindowInstance windowInstance, + String queueId) { + if (allValueList.size() <= 1) { + return allValueList; + } + Map deleteValueMap = new HashMap<>(allValueList.size()); + Map> mergeValueMap = new HashMap<>(allValueList.size()); + Collections.sort(allValueList, Comparator.comparing(WindowValue::getStartTime)); + for (int outIndex = 0; outIndex < allValueList.size(); outIndex++) { + if (deleteValueMap.containsKey(outIndex)) { + continue; + } + int finalOutIndex = outIndex; + mergeValueMap.put(outIndex, new ArrayList() {{ + add(finalOutIndex); + }}); + WindowValue outValue = allValueList.get(outIndex); + for (int inIndex = outIndex + 1; inIndex < allValueList.size(); inIndex++) { + WindowValue inValue = allValueList.get(inIndex); + if (inValue.getFireTime().compareTo(outValue.getEndTime()) <= 0) { + deleteValueMap.put(inIndex, outIndex); + outValue.setEndTime(outValue.getEndTime().compareTo(inValue.getEndTime()) <= 0 ? inValue.getEndTime() : outValue.getEndTime()); + outValue.setFireTime(outValue.getFireTime().compareTo(inValue.getFireTime()) <= 0 ? inValue.getFireTime() : outValue.getFireTime()); + mergeValueMap.get(outIndex).add(inIndex); + } else { + break; + } + } + } + Iterator>> iterator = mergeValueMap.entrySet().iterator(); + List resultList = new ArrayList<>(); + while (iterator.hasNext()) { + Map.Entry> entry = iterator.next(); + WindowValue theValue = allValueList.get(entry.getKey()); + List indexList = entry.getValue(); + WindowValue tempValue = WindowValue.mergeWindowValue(this, indexList.stream().map(index -> allValueList.get(index)).collect(Collectors.toList())); + theValue.setComputedColumnResult(tempValue.getComputedColumnResult()); + theValue.setAggColumnResult(tempValue.getAggColumnResult()); + resultList.add(theValue); + } + List prefixKeyList = deleteValueMap.keySet().stream().map(index -> createPrefixKey(allValueList.get(index), windowInstance, queueId)).collect(Collectors.toList()); + deletePrefixValue(prefixKeyList); + return resultList; + } + + private void deletePrefixValue(List keyList) { + if (CollectionUtil.isEmpty(keyList)) { + return; + } + storage.getLocalStorage().removeKeys(keyList); + } + + private static String createPrefixKey(WindowValue windowValue, WindowInstance windowInstance, String queueId) { + return MapKeyUtil.createKey(getOrderBypPrefix() + queueId, windowInstance.createWindowInstanceId(), windowValue.getFireTime(), String.valueOf(windowValue.getPartitionNum()), windowValue.getGroupBy()); + } + + private Pair getSessionTime(IMessage message) { + Long occurTime = System.currentTimeMillis(); + try { + occurTime = WindowInstance.getOccurTime(this, message); + } catch (Exception e) { + LOG.error("failed in computing occur time from the message!", e); + } + Date occurDate = new Date(occurTime); + Date endDate = DateUtil.addDate(TimeUnit.SECONDS, occurDate, sessionTimeOut); + return Pair.of(occurDate, endDate); + } + + protected void store(Map> key2ValueMap, WindowInstance windowInstance, + String queueId) { + // + if (CollectionUtil.isEmpty(key2ValueMap)) { + return; + } + // + storage.multiPutList(key2ValueMap, windowInstance.createWindowInstanceId(), queueId, sqlCache); + // + Map allValueMap = new HashMap<>(); + Iterator>> iterator = key2ValueMap.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry> entry = iterator.next(); + List valueList = entry.getValue(); + for (WindowValue value : valueList) { + String partitionNumKey = createPrefixKey(value, windowInstance, queueId); + allValueMap.put(partitionNumKey, value); + } + } + storage.getLocalStorage().multiPut(allValueMap); + } + + /** + * create new session window value + * + * @param queueId + * @param groupBy + * @param instance + * @return + */ + protected WindowValue createWindowValue(String queueId, String groupBy, WindowInstance instance, IMessage message, + String storeKey) { + WindowValue value = new WindowValue(); + value.setNameSpace(getNameSpace()); + value.setConfigureName(getConfigureName()); + Pair startEndPair = getSessionTime(message); + String startTime = DateUtil.format(startEndPair.getLeft(), SESSION_DATETIME_PATTERN); + String endTime = DateUtil.format(startEndPair.getRight(), SESSION_DATETIME_PATTERN); + String fireTime = DateUtil.format(DateUtil.addDate(TimeUnit.SECONDS, startEndPair.getRight(), waterMarkMinute * timeUnitAdjust), SESSION_DATETIME_PATTERN); + value.setStartTime(startTime); + value.setEndTime(endTime); + value.setFireTime(fireTime); + value.setGroupBy(groupBy); + value.setMsgKey(StringUtil.createMD5Str(storeKey)); + //FIXME shuffleId vs queueId TODO delete assert + String shuffleId = shuffleChannel.getChannelQueue(groupBy).getQueueId(); + assert shuffleId.equalsIgnoreCase(queueId); + value.setPartitionNum(createPartitionNum(value, queueId, instance)); + value.setPartition(shuffleId); + value.setWindowInstancePartitionId(instance.getWindowInstanceKey()); + value.setWindowInstanceId(instance.getWindowInstanceKey()); + return value; + } + + protected static String getOrderBypPrefix() { + return ORDER_BY_FIRE_TIME_PREFIX; + } + + /** + * update session's next fire time + * + * @param windowInstance + * @param queueId + * @param queueId2Offset + * @return + */ + @Override + public int fireWindowInstance(WindowInstance windowInstance, String queueId, Map queueId2Offset) { + synchronized (lock) { + //get iterator sorted by fire time + WindowBaseValueIterator it = storage.loadWindowInstanceSplitData(getOrderBypPrefix(), queueId, windowInstance.createWindowInstanceId(), null, getWindowBaseValueClass()); + // + if (queueId2Offset != null) { + String offset = queueId2Offset.get(queueId); + if (StringUtil.isNotEmpty(offset)) { + it.setPartitionNum(Long.valueOf(offset)); + } + } + // + Long currentFireTime = DateUtil.parse(windowInstance.getFireTime(), SESSION_DATETIME_PATTERN).getTime(); + Long nextFireTime = currentFireTime + 1000 * 60 * 1; + List toFireValueList = new ArrayList<>(); + while (it.hasNext()) { + WindowValue windowValue = it.next(); + if (windowValue == null) { + continue; + } + if (checkFire(queueId, windowValue)) { + TraceUtil.debug(String.valueOf(windowValue.getPartitionNum()), "shuffle message fire", windowValue.getStartTime(), windowValue.getEndTime(), windowValue.getComputedColumnResult()); + toFireValueList.add(windowValue); + } else { + Long itFireTime = DateUtil.parse(windowValue.getFireTime(), SESSION_DATETIME_PATTERN).getTime(); + if (itFireTime > currentFireTime && itFireTime < nextFireTime) { + nextFireTime = itFireTime; + break; + } + } + } + doFire(queueId, windowInstance, toFireValueList, currentFireTime, nextFireTime); + // + return toFireValueList.size(); + } + + } + + private boolean checkFire(String queueId, WindowValue value) { + Long maxEventTime = getMaxEventTime(queueId); + //set current time if not every queue have arrived + if (maxEventTime == null) { + maxEventTime = System.currentTimeMillis(); + } + Long fireTime = DateUtil.parse(value.getFireTime(), SESSION_DATETIME_PATTERN).getTime(); + if (fireTime < maxEventTime) { + return true; + } + return false; + } + + private void doFire(String queueId, WindowInstance instance, List valueList, Long currentFireTime, + Long nextFireTime) { + if (CollectionUtil.isEmpty(valueList)) { + return; + } + valueList.sort(Comparator.comparingLong(WindowBaseValue::getPartitionNum)); + sendFireMessage(valueList, queueId); + clearWindowValues(valueList, queueId, instance); + // + if (!nextFireTime.equals(currentFireTime)) { + String instanceId = instance.createWindowInstanceId(); + WindowInstance existedWindowInstance = searchWindowInstance(instanceId); + if (existedWindowInstance != null) { + existedWindowInstance.setFireTime(DateUtil.format(new Date(nextFireTime))); + windowFireSource.registFireWindowInstanceIfNotExist(instance, this); + } else { + LOG.error("window instance lost, queueId: " + queueId + " ,fire time" + instance.getFireTime()); + } + } + } + + /** + * clear fired window value + * + * @param deleteValueList + * @param queueId + * @param instance + */ + protected void clearWindowValues(List deleteValueList, String queueId, WindowInstance instance) { + if (CollectionUtil.isEmpty(deleteValueList)) { + return; + } + Set storeKeySet = new HashSet<>(deleteValueList.size()); + Set valueIdSet = new HashSet<>(deleteValueList.size()); + Set prefixKeySet = new HashSet<>(deleteValueList.size()); + for (WindowValue windowValue : deleteValueList) { + String storeKey = createStoreKey(queueId, windowValue.getGroupBy(), instance); + String prefixKey = createPrefixKey(windowValue, instance, queueId); + Long valueId = windowValue.getPartitionNum(); + storeKeySet.add(storeKey); + valueIdSet.add(valueId); + prefixKeySet.add(prefixKey); + } + Map> storeValueMap = storage.multiGetList(WindowValue.class, new ArrayList<>(storeKeySet)); + Map> lastValueMap = new HashMap<>(storeValueMap.size()); + Iterator>> iterator = storeValueMap.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry> entry = iterator.next(); + String storeKey = entry.getKey(); + List valueList = entry.getValue(); + valueList = valueList.stream().filter(value -> !valueIdSet.contains(value.getPartitionNum())).collect(Collectors.toList()); + lastValueMap.put(storeKey, valueList); + } + storage.getLocalStorage().removeKeys(prefixKeySet); + store(lastValueMap, instance, queueId); + } + + @Override + public long incrementAndGetSplitNumber(WindowInstance instance, String shuffleId) { + long numer = super.incrementAndGetSplitNumber(instance, shuffleId); + if (numer > 900000000) { + this.getWindowMaxValueManager().resetSplitNum(instance, shuffleId); + } + return numer; + } +} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionWindow.java deleted file mode 100644 index 2df603b0..00000000 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionWindow.java +++ /dev/null @@ -1,229 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.streams.window.operator.impl; - -import java.util.ArrayList; -import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.rocketmq.streams.common.context.IMessage; -import org.apache.rocketmq.streams.common.utils.DateUtil; -import org.apache.rocketmq.streams.common.utils.MapKeyUtil; -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.window.model.WindowInstance; -import org.apache.rocketmq.streams.window.state.WindowBaseValue; -import org.apache.rocketmq.streams.window.state.impl.WindowValue; -import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator; - -/** - * 实现思路: 1.每个分片一个windowinstance,starttime=2020-12-30:00:00:00 endtime=2999-01-01 01:01:01 2.firetime,第一次创建窗口,firetime=当前时间计算+window size 3.增加存储,按window instance所有groupby的触发时间排序(设计前缀),每次有数据来时,更新触发时间,触发时间算法如2 4.窗口触发时,检查存储中最近的触发时间是否<=触发时间,如果符合触发条件触发,然后一直迭代到触发时间>当前时间,把最近的触发时间当作window instance的触发时间,修改window instance的firetime 5.清理触发的数据(触发时间<=窗口实例的触发时间) - */ -public class SessionWindow extends WindowOperator { - private static final String ORDER_BY_FIRE_TIME = "_order_by_fire_time_ ";//key=_order;queueid,windowinstanceid,partitionNum - - @Override - protected boolean initConfigurable() { - this.fireMode = 2; - return super.initConfigurable(); - } - - @Override - public int fireWindowInstance(WindowInstance instance, String queueId, Map queueId2Offset) { - List fireWindowValues = new ArrayList<>(); - int fireCount = 0; - //for(String queueId:queueIds){ - WindowBaseValueIterator it = storage.loadWindowInstanceSplitData(getOrderBypPrefix(), queueId, instance.createWindowInstanceId(), null, getWindowBaseValueClass()); - if (queueId2Offset != null) { - String offset = queueId2Offset.get(queueId); - if (StringUtil.isNotEmpty(offset)) { - it.setPartitionNum(Long.valueOf(offset)); - } - } - boolean hasFinished = true; - while (it.hasNext()) { - WindowBaseValue windowBaseValue = it.next(); - if (windowBaseValue == null) { - continue; - } - Date realFireTime = DateUtil.parseTime(instance.getFireTime()); - Long currentMaxTime = instance.getLastMaxUpdateTime(); - Long realFireTimeLong = realFireTime.getTime(); - // System.out.println(DateUtil.format(new Date(currentMaxTime))); - /** - * first not fire window value - */ - if (currentMaxTime - realFireTimeLong < 0) { - instance.setFireTime(windowBaseValue.getFireTime()); - windowFireSource.registFireWindowInstanceIfNotExist(instance, this); - hasFinished = false; - break; - } - fireWindowValues.add((WindowValue)windowBaseValue); - if (fireWindowValues.size() >= windowCache.getBatchSize()) { - sendFireMessage(fireWindowValues, queueId); - fireCount += fireWindowValues.size(); - clearWindowValues(fireWindowValues, queueId, instance); - fireWindowValues = new ArrayList<>(); - } - - } - if (fireWindowValues.size() > 0) { - sendFireMessage(fireWindowValues, queueId); - fireCount += fireWindowValues.size(); - clearWindowValues(fireWindowValues, queueId, instance); - } - if (hasFinished) { - this.windowInstanceMap.remove(instance.createWindowInstanceId()); - } - - //} - - return fireCount; - } - - @Override - protected void saveStorage(Map allWindowBasedValue, WindowInstance windowInstance, String queueId) { - List oldKeys = new ArrayList<>(); - Map partionNumOrders = new HashMap<>();//需要基于key前缀排序partitionnum - for (WindowBaseValue windowBaseValue : allWindowBasedValue.values()) { - WindowValue windowValue = (WindowValue)windowBaseValue; - String oldPartitionNumKey = createStoreKey(getOrderBypPrefix() + queueId, MapKeyUtil.createKey(getOrderBypFieldName(windowValue), windowValue.getGroupBy()), windowInstance); - - windowBaseValue.setPartitionNum(createPartitionNum((WindowValue)windowBaseValue, queueId, windowInstance)); - windowBaseValue.setFireTime(createSessionFireTime(windowValue.getPartition(), windowValue.getLastUpdateTime())); - String partitionNumKey = createStoreKey(getOrderBypPrefix() + queueId, MapKeyUtil.createKey(getOrderBypFieldName(windowValue), windowValue.getGroupBy()), windowInstance); - if (!partitionNumKey.equals(oldPartitionNumKey)) { - oldKeys.add(oldPartitionNumKey); - partionNumOrders.put(partitionNumKey, windowValue); - } - - } - this.storage.getLocalStorage().removeKeys(oldKeys); - storage.multiPut(allWindowBasedValue); - storage.multiPut(partionNumOrders); - } - - @Override - public List queryOrCreateWindowInstance(IMessage message, String queueId) { - Long occurTime = WindowInstance.getOccurTime(this, message); - Date fireTime = createSessionFireDate(queueId, occurTime); - WindowInstance windowInstance = this.createWindowInstance("2020-01-01 00:00:00", "2999-01-01 00:00:00", DateUtil.format(fireTime), queueId); - WindowInstance existWindowInstance = this.getWindowInstanceMap().get(windowInstance.createWindowInstanceId()); - if (existWindowInstance != null) { - Date windowInstanceFireTime = DateUtil.parse(existWindowInstance.getFireTime()); - boolean hasFired = false; - while (WindowInstance.getOccurTime(this, message) - windowInstanceFireTime.getTime() > 0) { - hasFired = true; - System.out.println(DateUtil.format(new Date(WindowInstance.getOccurTime(this, message)))); - existWindowInstance.setLastMaxUpdateTime(WindowInstance.getOccurTime(this, message)); - this.windowFireSource.executeFireTask(existWindowInstance, true); - existWindowInstance = this.getWindowInstanceMap().get(windowInstance.createWindowInstanceId()); - if (existWindowInstance == null) { - break; - } - windowInstanceFireTime = DateUtil.parse(existWindowInstance.getFireTime()); - - } - if (existWindowInstance != null) { - windowInstance = existWindowInstance; - } - if (hasFired) { - windowInstance.setFireTime(DateUtil.format(fireTime)); - } - - } else { - windowInstance.setNewWindowInstance(true); - windowInstance.setFireTime(DateUtil.format(fireTime)); - this.getWindowInstanceMap().put(windowInstance.createWindowInstanceId(), windowInstance); - } - List windowInstances = new ArrayList<>(); - windowInstances.add(windowInstance); - return windowInstances; - } - - /** - * clear has fired window value - * - * @param windowValues - * @param queueId - * @param instance - */ - protected void clearWindowValues(List windowValues, String queueId, WindowInstance instance) { - if (windowValues == null || windowValues.size() == 0) { - return; - } - Set deleteKeys = new HashSet<>(); - List msgKeys = new ArrayList<>(); - for (WindowValue windowValue : windowValues) { - String storeKey = createStoreKey(queueId, windowValue.getGroupBy(), instance); - String partitionNumKey = createStoreKey(getOrderBypPrefix() + queueId, MapKeyUtil.createKey(getOrderBypFieldName(windowValue), windowValue.getGroupBy()) + "", instance); - deleteKeys.add(storeKey); - deleteKeys.add(partitionNumKey); - msgKeys.add(windowValue.getMsgKey()); - } - String sql = "delete from window_value where msg_key in(" + SQLUtil.createInSql(msgKeys) + ")"; - DriverBuilder.createDriver().execute(sql); - storage.getLocalStorage().removeKeys(deleteKeys); - } - - protected static String getOrderBypFieldName(WindowValue windowValue) { - return MapKeyUtil.createKey(windowValue.getFireTime(), windowValue.getPartitionNum() + ""); - } - - /** - * create min session fire time, the current time+window size - * - * @param splitId - * @param occurTime - * @return - */ - protected String createSessionFireTime(String splitId, Long occurTime) { - Date newFireTime = createSessionFireDate(splitId, occurTime); - return DateUtil.format(newFireTime); - } - - - /** - * create min session fire time, the current time+window size - * - * @param splitId - * @param lastUpdateTime - * @return - */ - protected Date createSessionFireDate(String splitId, Long lastUpdateTime) { - if (lastUpdateTime == null) { - // lastUpdateTime = this.updateMaxEventTime(splitId, (Long)null); - } - Date currentDate = new Date(lastUpdateTime); - Date newFireTime = DateUtil.addSecond(currentDate, this.sizeInterval * this.timeUnitAdjust); - return newFireTime; - } - - @Override - public long incrementAndGetSplitNumber(WindowInstance instance, String shuffleId) { - long numer = super.incrementAndGetSplitNumber(instance, shuffleId); - if (numer > 900000000) { - this.getWindowMaxValueManager().resetSplitNum(instance, shuffleId); - } - return numer; - } -} diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java index 0f573e27..c8de5c39 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java @@ -353,8 +353,7 @@ public void clearFireWindowInstance(WindowInstance windowInstance) { } if(canClear){ - this.windowInstanceMap.remove(windowInstance.createWindowInstanceTriggerId()); - + logoutWindowInstance(windowInstance.createWindowInstanceTriggerId()); windowMaxValueManager.deleteSplitNum(windowInstance,windowInstance.getSplitId()); ShufflePartitionManager.getInstance().clearWindowInstance(windowInstance.createWindowInstanceId()); storage.delete(windowInstance.createWindowInstanceId(),windowInstance.getSplitId(),getWindowBaseValueClass(),sqlCache); diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java index 1ea07b38..57c07f1c 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java @@ -20,12 +20,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.lang3.tuple.MutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.rocketmq.streams.common.context.IMessage; import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; @@ -34,10 +31,7 @@ import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.offset.WindowMaxValue; import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow; -import org.apache.rocketmq.streams.window.operator.AbstractWindow; -import org.apache.rocketmq.streams.window.sqlcache.impl.SQLElement; import org.apache.rocketmq.streams.window.sqlcache.impl.SplitSQLElement; -import org.apache.rocketmq.streams.window.state.WindowBaseValue; /** * @@ -113,13 +107,20 @@ protected String generateShuffleKey(IMessage message) { * @param instance2Messages * @param windowInstanceMap */ - protected void groupByWindowInstanceAndQueueId(List messageList, Map, List> instance2Messages, + protected void groupByWindowInstanceAndQueueId(List messageList, + Map, List> instance2Messages, Map windowInstanceMap) { for (IMessage message : messageList) { - - List windowInstances = (List)message.getMessageBody().get(WindowInstance.class.getSimpleName()); + //the queueId will be replace below, so get first here! String queueId = message.getHeader().getQueueId(); - for(WindowInstance windowInstance:windowInstances){ + String oriQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID); + String oriOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET); + Boolean isLong = message.getMessageBody().getBoolean(WindowCache.ORIGIN_QUEUE_IS_LONG); + message.getHeader().setQueueId(oriQueueId); + message.getHeader().setOffset(oriOffset); + message.getHeader().setOffsetIsLong(isLong); + List windowInstances = (List) message.getMessageBody().get(WindowInstance.class.getSimpleName()); + for (WindowInstance windowInstance : windowInstances) { String windowInstanceId = windowInstance.createWindowInstanceId(); Pair queueIdAndInstanceKey = Pair.of(queueId, windowInstanceId); List messages = instance2Messages.get(queueIdAndInstanceKey); @@ -127,20 +128,17 @@ protected void groupByWindowInstanceAndQueueId(List messageList, Map

(); instance2Messages.put(queueIdAndInstanceKey, messages); } - messages.add(message); + //in case of changing message concurrently in hop window + IMessage cloneMessage = message.deepCopy(); + //bring window instance id into accumulator computation + cloneMessage.getMessageBody().put("HIT_WINDOW_INSTANCE_ID", windowInstance.createWindowInstanceId()); + messages.add(cloneMessage); windowInstanceMap.put(windowInstanceId, windowInstance); } - - String oriQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID); - String oriOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET); - Boolean isLong = message.getMessageBody().getBoolean(WindowCache.ORIGIN_QUEUE_IS_LONG); - message.getHeader().setQueueId(oriQueueId); - message.getHeader().setOffset(oriOffset); - message.getHeader().setOffsetIsLong(isLong); - } } + // // public synchronized void addNeedFlushWindowInstance(WindowInstance windowInstance){ // if(!window.isLocalStorageOnly()){ 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 ba17a442..6f5590ea 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 @@ -18,6 +18,7 @@ import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; +import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -209,7 +210,7 @@ public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessa Map> queueId2WindowInstances = new HashMap<>(); for (WindowInstance windowInstance : allWindowInstances) { windowInstance.setNewWindowInstance(false); - window.getWindowInstanceMap().putIfAbsent(windowInstance.createWindowInstanceTriggerId(), windowInstance); + window.registerWindowInstance(windowInstance); window.getWindowFireSource().registFireWindowInstanceIfNotExist(windowInstance, window); String queueId = windowInstance.getSplitId(); window.getStorage().loadSplitData2Local(queueId, windowInstance.createWindowInstanceId(), window.getWindowBaseValueClass(), new WindowRowOperator(windowInstance, queueId, window)); @@ -383,23 +384,29 @@ public ISplit getSplit(Integer index) { } public JSONObject createMsg(JSONArray messages, ISplit split) { - JSONObject msg = new JSONObject(); - - msg.put(SHUFFLE_QUEUE_ID, split.getQueueId());//分片id - msg.put(SHUFFLE_MESSAGES, messages);//合并的消息 - msg.put(MSG_OWNER, getDynamicPropertyValue());//消息owner - - StringBuilder traceIds = new StringBuilder(); - for (int i = 0; i < messages.size(); i++) { - JSONObject object = messages.getJSONObject(i); - if (object.containsKey(WindowCache.ORIGIN_MESSAGE_TRACE_ID)) { - traceIds.append(object.getString(WindowCache.ORIGIN_MESSAGE_TRACE_ID)).append(";"); + //分片id + msg.put(SHUFFLE_QUEUE_ID, split.getQueueId()); + //合并的消息 + msg.put(SHUFFLE_MESSAGES, messages); + //消息owner + msg.put(MSG_OWNER, getDynamicPropertyValue()); + // + try { + List traceList = new ArrayList<>(); + List groupByList = new ArrayList<>(); + for (int i = 0; i < messages.size(); i++) { + JSONObject object = messages.getJSONObject(i); + groupByList.add(object.getString("SHUFFLE_KEY")); + traceList.add(object.getJSONObject("MessageHeader").getString("traceId")); } + String traceInfo = StringUtils.join(traceList); + String groupInfo = StringUtils.join(groupByList); + msg.put(SHUFFLE_TRACE_ID, StringUtils.join(traceList)); + TraceUtil.debug(traceInfo, "origin message out", split.getQueueId(), groupInfo, getConfigureName()); + } catch (Exception e) { + //do nothing } - msg.put(SHUFFLE_TRACE_ID, traceIds); - TraceUtil.debug(traceIds.toString(), "origin message out", split.getQueueId()); - return msg; } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/source/WindowRireSource.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/source/WindowFireSource.java similarity index 92% rename from rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/source/WindowRireSource.java rename to rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/source/WindowFireSource.java index c62915f1..8a7790e0 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/source/WindowRireSource.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/source/WindowFireSource.java @@ -40,10 +40,12 @@ import org.apache.rocketmq.streams.window.debug.DebugWriter; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractWindow; +import org.apache.rocketmq.streams.window.operator.impl.SessionOperator; -public class WindowRireSource extends AbstractSupportOffsetResetSource implements IStreamOperator { - protected static final Log LOG = LogFactory.getLog(WindowRireSource.class); +public class WindowFireSource extends AbstractSupportOffsetResetSource implements IStreamOperator { + protected static final Log LOG = LogFactory.getLog(WindowFireSource.class); private AbstractWindow window; + //TODO maxEventTime和fireTime都是相对时间,但是这个更新时间是绝对时间,使用起来会很别扭 protected transient Long eventTimeLastUpdateTime; protected transient ScheduledExecutorService fireCheckScheduler;//检查是否触发 protected transient ScheduledExecutorService checkpointScheduler; @@ -56,7 +58,7 @@ public class WindowRireSource extends AbstractSupportOffsetResetSource implement //> protected transient ConcurrentHashMap> windowInstanceQueueOffsets=new ConcurrentHashMap<>(); - public WindowRireSource(AbstractWindow window){ + public WindowFireSource(AbstractWindow window){ this.window=window; } @@ -108,8 +110,13 @@ public int compare(WindowInstance o1, WindowInstance o2) { } }); WindowInstance windowInstance = windowInstanceList.get(0); + //TODO 每一秒执行一个循环 while (windowInstance!=null){ - boolean success= executeFireTask(windowInstance,false); + boolean isStartNow = false; + if (SessionOperator.SESSION_WINDOW_BEGIN_TIME.equalsIgnoreCase(windowInstance.getStartTime())) { + isStartNow = true; + } + boolean success= executeFireTask(windowInstance,isStartNow); if(success){ windowInstances.remove(windowInstance.createWindowInstanceTriggerId()); } @@ -133,13 +140,12 @@ public int compare(WindowInstance o1, WindowInstance o2) { } windowInstance=windowInstanceList.get(0); } - } catch (Exception e) { e.printStackTrace(); } } - },0,1, TimeUnit.SECONDS); + }, 10, 1, TimeUnit.SECONDS); //定时发送checkpoint,提交和保存数据。在pull模式会有用 //fireCheckScheduler.scheduleWithFixedDelay(new Runnable() { @@ -175,7 +181,7 @@ public void registFireWindowInstanceIfNotExist(WindowInstance windowInstance, Ab String windowInstanceTriggerId=windowInstance.createWindowInstanceTriggerId(); WindowInstance old= windowInstances.putIfAbsent(windowInstanceTriggerId,windowInstance); if(old==null){ - window.getWindowInstanceMap().put(windowInstanceTriggerId,windowInstance); + window.registerWindowInstance(windowInstance); } LOG.debug("register window instance into manager, instance key: " + windowInstanceTriggerId); } @@ -267,7 +273,8 @@ protected FireResult canFire(WindowInstance windowInstance) { */ Long maxEventTime=this.window.getMaxEventTime(windowInstance.getSplitId()); if(maxEventTime==null){ - return new FireResult(); + //TODO + maxEventTime = System.currentTimeMillis(); } if(maxEventTime-fireTime.getTime()>=3000){ return new FireResult(true,0); @@ -277,10 +284,12 @@ protected FireResult canFire(WindowInstance windowInstance) { if(eventTimeLastUpdateTime==null){ return new FireResult(); } - int gap=(int)(System.currentTimeMillis()-eventTimeLastUpdateTime); - if(window.getMsgMaxGapSecond()!=null&&gap>window.getMsgMaxGapSecond()*1000){ - LOG.warn("the fire reason is exceed the gap "+gap+" window instance id is "+windowInstanceTriggerId); - return new FireResult(true,1); + if (isTest) { + int gap = (int) (System.currentTimeMillis() - eventTimeLastUpdateTime); + if (window.getMsgMaxGapSecond() != null && gap > window.getMsgMaxGapSecond() * 1000) { + LOG.warn("the fire reason is exceed the gap " + gap + " window instance id is " + windowInstanceTriggerId); + return new FireResult(true, 1); + } } return new FireResult(); } 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 e04bd161..6a595161 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 @@ -49,7 +49,6 @@ import org.apache.rocketmq.streams.script.operator.impl.FunctionScript; import org.apache.rocketmq.streams.script.service.IAccumulator; import org.apache.rocketmq.streams.window.model.FunctionExecutor; -import org.apache.rocketmq.streams.window.model.WindowCache; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.operator.AbstractWindow; import org.apache.rocketmq.streams.window.state.WindowBaseValue; @@ -262,10 +261,6 @@ public boolean calculate(AbstractWindow window, IMessage message) { } calFunctionColumn(window, message); calProjectColumn(window, message); - String traceId = message.getMessageBody().getString(WindowCache.ORIGIN_MESSAGE_TRACE_ID); - if (!StringUtil.isEmpty(traceId)) { - TraceUtil.debug(traceId, "window value result", decodeSQLContent(getComputedColumnResult())); - } } catch (Exception e) { LOG.error("failed in calculating the message", e); } @@ -380,10 +375,9 @@ public static List queryMergeWindowValues(AbstractWindow window, Li /** * merge the group which has the same group by value and different split id */ - private static WindowValue mergeWindowValue(AbstractWindow window, List valueList) { + public static WindowValue mergeWindowValue(AbstractWindow window, List valueList) { WindowValue lastWindowValue = new WindowValue(valueList.get(0)); - lastWindowValue.computedColumnResult = (Map)JSON.parse( - valueList.get(0).getComputedColumnResult()); + lastWindowValue.setComputedColumnResult(valueList.get(0).getComputedColumnResult()); // for (Entry> entry : window.getColumnExecuteMap().entrySet()) { String computedColumn = entry.getKey(); @@ -497,17 +491,6 @@ public WindowValue clone() { return clonedValue; } - public WindowValue toMd5Value() { - WindowValue clonedValue = clone(); - String md5MsgKey = StringUtil.createMD5Str(getMsgKey()); - clonedValue.setMsgKey(md5MsgKey); - clonedValue.setWindowInstanceId(StringUtil.createMD5Str(clonedValue.getWindowInstanceId())); - clonedValue.setWindowInstancePartitionId( - StringUtil.createMD5Str(clonedValue.getWindowInstancePartitionId())); - return clonedValue; - } - - public Long getLastUpdateTime() { return lastUpdateTime; } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ICommonStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ICommonStorage.java index 95003ac8..44a83feb 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ICommonStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ICommonStorage.java @@ -36,9 +36,25 @@ public interface ICommonStorage { //多组key value批量存储 void multiPut(Map map); + /** + * put into the storage + * + * @param elements + */ + void multiPutList(Map> elements); + //获取多个key的值 Map multiGet(Class clazz, List keys); + /** + * get list from storage according by key + * + * @param clazz + * @param keys + * @return + */ + Map> multiGetList(Class clazz, List keys); + //获取单个key的值 T get(Class clazz, String key); diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IRemoteStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IRemoteStorage.java index 807617ad..f7e93622 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IRemoteStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IRemoteStorage.java @@ -17,15 +17,20 @@ package org.apache.rocketmq.streams.window.storage; +import java.util.List; import java.util.Map; -import java.util.Set; public interface IRemoteStorage extends ICommonStorage { //多组key value批量存储 String multiPutSQL(Map values); - + /** + * generate sql sentence for session window + * @param infoMap + * @return + */ + String multiPutListSQL(Map> infoMap); String deleteSQL(String windowInstanceId, String queueId, Class clazz); } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java index 529162e7..a798fd4e 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java @@ -23,17 +23,25 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.common.utils.SQLUtil; +import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.sqlcache.SQLCache; import org.apache.rocketmq.streams.window.sqlcache.impl.SQLElement; import org.apache.rocketmq.streams.window.state.WindowBaseValue; +import org.apache.rocketmq.streams.window.state.impl.WindowValue; import org.apache.rocketmq.streams.window.storage.db.DBStorage; import org.apache.rocketmq.streams.window.storage.rocksdb.RocksdbStorage; @@ -84,6 +92,8 @@ public void multiPut(Map values, String windowInstanceId, String queu multiPut(values,windowInstanceId,queueId,null); } + + public void multiPut(Map values, String windowInstanceId, String queueId, SQLCache sqlCache) { localStorage.multiPut(values); if (isLocalStorageOnly) { @@ -103,6 +113,40 @@ public void multiPut(Map values, String windowInstanceId, String queu remoteStorage.multiPut(values); } + /** + * used in session window only + * + * @param values + * @param windowInstanceId + * @param queueId + * @param sqlCache + */ + public void multiPutList(Map> values, String windowInstanceId, String queueId, SQLCache sqlCache) { + localStorage.multiPutList(values); + if (!isLocalStorageOnly) { + //delete all values first + deleteRemoteValue(values.keySet()); + // + if (shufflePartitionManager.isWindowInstanceFinishInit(queueId, windowInstanceId)) { + if (sqlCache != null) { + sqlCache.addCache(new SQLElement(queueId, windowInstanceId, ((IRemoteStorage) this.remoteStorage).multiPutListSQL(values))); + } else { + remoteStorage.multiPutList(values); + } + return; + } + remoteStorage.multiPutList(values); + } + } + + private void deleteRemoteValue(Set storeKeyList) { + if (CollectionUtil.isEmpty(storeKeyList)) { + return; + } + String sql = "delete from " + ORMUtil.getTableName(WindowValue.class) + " where " + SQLUtil.createLikeSql(storeKeyList.stream().map(key -> Pair.of("msg_key", StringUtil.createMD5Str(key))).collect(Collectors.toList())); + ORMUtil.executeSQL(sql, new HashMap<>(4)); + } + @Override public Long getMaxSplitNum(WindowInstance windowInstance, Class clazz) { if(isLocalStorageOnly){ return null; @@ -118,6 +162,40 @@ public Map multiGet(Class clazz, List keys, String windowI return remoteStorage.multiGet(clazz, keys); } + @Override public void multiPutList(Map> elements) { + if (!isLocalStorageOnly) { + remoteStorage.multiPutList(elements); + } + localStorage.multiPutList(elements); + } + + @Override public Map> multiGetList(Class clazz, List keys) { + if (isLocalStorageOnly) { + return localStorage.multiGetList(clazz, keys); + } + Map> resultMap = new HashMap<>(keys.size()); + Pair, List> pair = getStorageKeys(keys); + resultMap.putAll(localStorage.multiGetList(clazz, pair.getLeft())); + resultMap.putAll(remoteStorage.multiGetList(clazz, pair.getRight())); + return resultMap; + } + + private Pair, List> getStorageKeys(List allKeys) { + List remoteKeys = new ArrayList<>(); + List localKeys = new ArrayList<>(); + for (String key : allKeys) { + String[] values = MapKeyUtil.spliteKey(key); + String shuffleId = values[0]; + boolean isLocal = shufflePartitionManager.isWindowInstanceFinishInit(shuffleId, createWindowInstanceId(key)); + if (isLocal) { + localKeys.add(key); + } else { + remoteKeys.add(key); + } + } + return Pair.of(localKeys, remoteKeys); + } + @Override public void multiPut(Map values) { localStorage.multiPut(values); @@ -135,22 +213,9 @@ public Map multiGet(Class clazz, List keys) { result.putAll(localStorage.multiGet(clazz, keys)); return result; } - - List notLocalKeys = new ArrayList<>(); - List localKeys = new ArrayList<>(); - for (String key : keys) { - String[] values = MapKeyUtil.spliteKey(key); - String shuffleId = values[0]; - boolean isLocal = shufflePartitionManager.isWindowInstanceFinishInit(shuffleId, createWindowInstanceId(key)); - if (isLocal) { - localKeys.add(key); - } else { - notLocalKeys.add(key); - } - } - - result.putAll(localStorage.multiGet(clazz, localKeys)); - result.putAll(remoteStorage.multiGet(clazz, notLocalKeys)); + Pair, List> pair = getStorageKeys(keys); + result.putAll(localStorage.multiGet(clazz, pair.getLeft())); + result.putAll(remoteStorage.multiGet(clazz, pair.getRight())); return result; } diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java index b0fe5fe8..dfdf2618 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java @@ -19,9 +19,11 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; +import org.apache.commons.lang3.tuple.Pair; import org.apache.rocketmq.streams.common.channel.split.ISplit; import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; @@ -30,6 +32,7 @@ import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.state.WindowBaseValue; +import org.apache.rocketmq.streams.window.state.impl.WindowValue; import org.apache.rocketmq.streams.window.storage.AbstractWindowStorage; import org.apache.rocketmq.streams.window.storage.IRemoteStorage; import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator; @@ -47,12 +50,41 @@ public class DBStorage extends AbstractWindowStorage< return sql; } + @Override public String multiPutListSQL(Map> infoMap) { + if (CollectionUtil.isNotEmpty(infoMap)) { + List valueList = duplicate(infoMap); + return ORMUtil.createBatchReplacetSQL(valueList); + } + return null; + } + + /** + * the list value has the same store key, add suffix for session window + * + * @param infoMap + * @return + */ + private List duplicate(Map> infoMap) { + List resultList = new ArrayList<>(); + Iterator>> iterator = infoMap.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry> entry = iterator.next(); + List valueList = entry.getValue(); + for (int index = 0; index < valueList.size(); index++) { + //TODO 是否要进行clone + T value = valueList.get(index); + value.setMsgKey(value.getMsgKey() + "_" + index); + resultList.add(value); + } + } + return resultList; + } + @Override public void multiPut(Map values) { if (CollectionUtil.isEmpty(values)) { return; } - ORMUtil.batchReplaceInto(values.values()); } @@ -78,6 +110,51 @@ public Map multiGet(Class clazz, List keys) { return map; } + @Override public void multiPutList(Map> elements) { + if (CollectionUtil.isEmpty(elements)) { + return; + } + List valueList = duplicate(elements); + ORMUtil.batchReplaceInto(valueList); + } + + /** + * the key in db is md5(key)_index + * + * @param clazz + * @param keys + * @return + */ + @Override public Map> multiGetList(Class clazz, List keys) { + if (CollectionUtil.isEmpty(keys)) { + return new HashMap<>(4); + } + Map recordMap = new HashMap<>(keys.size()); + List dbKeyList = new ArrayList<>(keys.size()); + List> variableAndValue = new ArrayList<>(keys.size()); + for (String key : keys) { + String md5Value = StringUtil.createMD5Str(key); + dbKeyList.add(md5Value); + recordMap.put(md5Value, key); + variableAndValue.add(Pair.of("msg_key", md5Value + "%")); + } + List values = ORMUtil.queryForList("select * from " + ORMUtil.getTableName(clazz) + + " where "+SQLUtil.createLikeSql(variableAndValue), new HashMap<>(4), clazz); + Map> resultMap = new HashMap<>(keys.size()); + for (T value : values) { + String dbKeyWithoutSuffix = value.getMsgKey().substring(0, 24); + value.setMsgKey(dbKeyWithoutSuffix); + String key = recordMap.get(dbKeyWithoutSuffix); + List valueList = resultMap.getOrDefault(key, null); + if (valueList == null) { + valueList = new ArrayList<>(); + resultMap.put(key, valueList); + } + valueList.add(value); + } + return resultMap; + } + @Override public void removeKeys(Collection keys) { @@ -131,6 +208,8 @@ public void delete(String windowInstanceId, String queueId, Class clazz) { return sql; } + + public static class DBIterator extends WindowBaseValueIterator { private LinkedList container = new LinkedList<>(); int batchSize = 1000; diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/file/FileStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/file/FileStorage.java index 683a84b6..bba64a81 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/file/FileStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/file/FileStorage.java @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.streams.window.storage.file; +import com.alibaba.fastjson.JSONArray; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -104,6 +105,19 @@ public synchronized void multiPut(Map map) { this.flush(); } + @Override public synchronized void multiPutList(Map> elements) { + for (Entry> entry : elements.entrySet()) { + String key = entry.getKey(); + List valueList = entry.getValue(); + JSONArray array = new JSONArray(); + for (T value : valueList) { + array.add(value.toJsonObject()); + } + this.cache.put(key, array.toJSONString()); + } + this.flush(); + } + @Override public synchronized Map multiGet(Class clazz, List keys) { Map result = new HashMap<>(); @@ -118,6 +132,25 @@ public synchronized Map multiGet(Class clazz, List keys) { return result; } + @Override public synchronized Map> multiGetList(Class clazz, List keys) { + Map> result = new HashMap<>(); + for (String key : keys) { + String value = this.cache.get(key); + if (StringUtil.isNotEmpty(value)) { + JSONArray jsonArray = JSONArray.parseArray(value); + List valueList = new ArrayList<>(); + for (int index = 0; index < jsonArray.size(); index++) { + String objectString = jsonArray.getString(index); + T valueObject = ReflectUtil.forInstance(clazz); + valueObject.toObject(objectString); + valueList.add(valueObject); + } + result.put(key, valueList); + } + } + return result; + } + @Override public synchronized void removeKeys(Collection keys) { for (String key : keys) { diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java index 5bb9ff1a..3bb98fbe 100644 --- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java +++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.streams.window.storage.rocksdb; +import com.alibaba.fastjson.JSONArray; import java.io.File; import java.io.UnsupportedEncodingException; import java.util.ArrayList; @@ -29,11 +30,13 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.rocketmq.streams.common.channel.split.ISplit; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.FileUtil; import org.apache.rocketmq.streams.common.utils.MapKeyUtil; import org.apache.rocketmq.streams.common.utils.ReflectUtil; import org.apache.rocketmq.streams.common.utils.RuntimeUtil; import org.apache.rocketmq.streams.common.utils.StringUtil; +import org.apache.rocketmq.streams.state.kv.rocksdb.RocksDBOperator; import org.apache.rocketmq.streams.window.model.WindowInstance; import org.apache.rocketmq.streams.window.state.WindowBaseValue; import org.apache.rocketmq.streams.window.storage.AbstractWindowStorage; @@ -51,45 +54,10 @@ public class RocksdbStorage extends AbstractWindowSto protected static String DB_PATH = "/tmp/rocksdb"; protected static String UTF8 = "UTF8"; protected static AtomicBoolean hasCreate = new AtomicBoolean(false); - protected static RocksDB rocksDB; + protected static RocksDB rocksDB = new RocksDBOperator().getInstance(); protected WriteOptions writeOptions = new WriteOptions(); - static { - RocksDB.loadLibrary(); - } - - public RocksdbStorage() { - this(FileUtil.concatFilePath(StringUtil.isEmpty(FileUtil.getJarPath()) ? DB_PATH + File.separator + RuntimeUtil.getDipperInstanceId() : FileUtil.getJarPath() + File.separator + RuntimeUtil.getDipperInstanceId(), "rocksdb")); - } - - public RocksdbStorage(String rocksdbFilePath) { - if (hasCreate.compareAndSet(false, true)) { - synchronized (RocksdbStorage.class) { - if (RocksdbStorage.rocksDB == null) { - synchronized (RocksdbStorage.class) { - if (RocksdbStorage.rocksDB == null) { - try (final Options options = new Options().setCreateIfMissing(true)) { - try { - File dir = new File(rocksdbFilePath); - if (dir.exists()) { - dir.delete(); - } - dir.mkdirs(); - final TtlDB db = TtlDB.open(options, rocksdbFilePath, 10800, false); - RocksdbStorage.rocksDB = db; - writeOptions.setSync(true); - } catch (RocksDBException e) { - throw new RuntimeException("create rocksdb error " + e.getMessage()); - } - } - } - } - } - } - } - - } @Override public void removeKeys(Collection keys) { @@ -177,6 +145,69 @@ public Map multiGet(Class clazz, List keys) { } + @Override public void multiPutList(Map> elements) { + if (CollectionUtil.isEmpty(elements)) { + return; + } + try { + WriteBatch writeBatch = new WriteBatch(); + Iterator>> it = elements.entrySet().iterator(); + while (it.hasNext()) { + Entry> entry = it.next(); + String key = entry.getKey(); + List valueList = entry.getValue(); + JSONArray array = new JSONArray(); + for (T value : valueList) { + array.add(value.toJsonObject()); + } + writeBatch.put(key.getBytes(UTF8), array.toJSONString().getBytes(UTF8)); + } + WriteOptions writeOptions = new WriteOptions(); + writeOptions.setSync(false); + writeOptions.setDisableWAL(true); + rocksDB.write(writeOptions, writeBatch); + writeBatch.close(); + writeOptions.close(); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("put data to rocksdb error", e); + } + } + + @Override public Map> multiGetList(Class clazz, List keys) { + if (CollectionUtil.isEmpty(keys)) { + return new HashMap<>(4); + } + List keyByteList = new ArrayList<>(); + for (String key : keys) { + keyByteList.add(getKeyBytes(key)); + } + try { + Map> resultMap = new HashMap<>(); + Map map = rocksDB.multiGet(keyByteList); + int i = 0; + Iterator> it = map.entrySet().iterator(); + while (it.hasNext()) { + Entry entry = it.next(); + String key = getValueFromByte(entry.getKey()); + String value = getValueFromByte(entry.getValue()); + JSONArray array = JSONArray.parseArray(value); + List valueList = new ArrayList<>(); + for (int index = 0; index < array.size(); index++) { + String objectString = array.getString(index); + T valueObject = ReflectUtil.forInstance(clazz); + valueObject.toObject(objectString); + valueList.add(valueObject); + } + resultMap.put(key, valueList); + } + return resultMap; + } catch (RocksDBException e) { + e.printStackTrace(); + throw new RuntimeException("can not get multi value from rocksdb! ", e); + } + } + @Override public void clearCache(ISplit split, Class clazz) { deleteRange(split.getQueueId(), split.getPlusQueueId(), clazz); diff --git a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/storage/RocksdbTest.java b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/storage/RocksdbTest.java new file mode 100644 index 00000000..c708fdd3 --- /dev/null +++ b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/storage/RocksdbTest.java @@ -0,0 +1,190 @@ +package org.apache.rocketmq.streams.storage; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.rocketmq.streams.common.utils.MapKeyUtil; +import org.apache.rocketmq.streams.window.model.WindowInstance; +import org.apache.rocketmq.streams.window.operator.impl.SessionOperator; +import org.apache.rocketmq.streams.window.state.WindowBaseValue; +import org.apache.rocketmq.streams.window.state.impl.WindowValue; +import org.apache.rocketmq.streams.window.storage.WindowStorage; +import org.apache.rocketmq.streams.window.storage.rocksdb.RocksdbStorage; +import org.junit.Assert; +import org.junit.Test; + +public class RocksdbTest { + + private static RocksdbStorage storage = new RocksdbStorage<>(); + + @Test + public void testMultiProcess() { + // + RocksdbStorage storage1 = new RocksdbStorage(); + RocksdbStorage storage2 = new RocksdbStorage(); + // + // + WindowBaseValue value1 = new WindowBaseValue(); + value1.setStartTime("2021-09-07 11:00:00"); + value1.setEndTime("2021-09-07 11:10:00"); + value1.setFireTime("2021-09-07 11:11:00"); + WindowBaseValue value2 = new WindowBaseValue(); + value2.setStartTime("2021-09-07 12:00:00"); + value2.setEndTime("2021-09-07 12:10:00"); + value2.setFireTime("2021-09-07 12:11:00"); + // + storage1.put("storage_1", value1); + storage2.put("storage_2", value2); + // + RocksdbStorage storage3 = new RocksdbStorage(); + Map valueMap = storage3.multiGet(WindowBaseValue.class, new ArrayList() {{ + add("storage_1"); + add("storage_2"); + }}); + Assert.assertEquals(2, valueMap.size()); + } + + @Test + public void testMultiValues() { + // + List valueList = new ArrayList<>(); + WindowBaseValue value1 = new WindowBaseValue(); + value1.setStartTime("2021-09-07 11:00:00"); + value1.setEndTime("2021-09-07 11:10:00"); + value1.setFireTime("2021-09-07 11:11:00"); + WindowBaseValue value2 = new WindowBaseValue(); + value2.setStartTime("2021-09-07 12:00:00"); + value2.setEndTime("2021-09-07 12:10:00"); + value2.setFireTime("2021-09-07 12:11:00"); + valueList.add(value1); + valueList.add(value2); + // + String key = "test"; + Map> theMap = new HashMap<>(); + theMap.put(key, valueList); + storage.multiPutList(theMap); + Map> resultMap = storage.multiGetList(WindowBaseValue.class, new ArrayList() {{ + add(key); + }}); + Assert.assertEquals(1, resultMap.size()); + Assert.assertEquals(2, resultMap.get(key).size()); + Assert.assertEquals("2021-09-07 11:00:00", resultMap.get(key).get(0).getStartTime()); + Assert.assertEquals("2021-09-07 12:00:00", resultMap.get(key).get(1).getStartTime()); + // + storage.removeKeys(theMap.keySet()); + } + + @Test + public void testOverwrite() { + // + List valueList = new ArrayList<>(); + WindowBaseValue value1 = new WindowBaseValue(); + value1.setStartTime("2021-09-07 11:00:00"); + value1.setEndTime("2021-09-07 11:10:00"); + value1.setFireTime("2021-09-07 11:11:00"); + WindowBaseValue value2 = new WindowBaseValue(); + value2.setStartTime("2021-09-07 12:00:00"); + value2.setEndTime("2021-09-07 12:10:00"); + value2.setFireTime("2021-09-07 12:11:00"); + valueList.add(value1); + valueList.add(value2); + // + String key = "test"; + Map> theMap = new HashMap<>(); + theMap.put(key, valueList); + storage.multiPutList(theMap); + // + valueList = new ArrayList<>(); + value1 = new WindowBaseValue(); + value1.setStartTime("2021-09-11 11:00:00"); + value1.setEndTime("2021-09-11 11:10:00"); + value1.setFireTime("2021-09-11 11:11:00"); + valueList.add(value1); + theMap.put(key, valueList); + storage.multiPutList(theMap); + Map> resultMap = storage.multiGetList(WindowBaseValue.class, new ArrayList() {{ + add(key); + }}); + Assert.assertEquals(1, resultMap.size()); + Assert.assertEquals("2021-09-11 11:00:00", resultMap.get(key).get(0).getStartTime()); + // + storage.removeKeys(resultMap.keySet()); + } + + @Test + public void testValueWithPrefix() { + // + WindowInstance windowInstance = new WindowInstance(); + windowInstance.setStartTime(SessionOperator.SESSION_WINDOW_BEGIN_TIME); + windowInstance.setEndTime(SessionOperator.SESSION_WINDOW_END_TIME); + windowInstance.setFireTime("2021-09-07 12:00:00"); + // + Map valueMap = new HashMap<>(); + WindowValue value1 = new WindowValue(); + value1.setStartTime("2021-09-07 11:00:00"); + value1.setEndTime("2021-09-07 11:10:00"); + value1.setFireTime("2021-09-07 11:11:00"); + value1.setPartitionNum(100001); + WindowValue value2 = new WindowValue(); + value2.setStartTime("2021-09-07 12:00:00"); + value2.setEndTime("2021-09-07 12:10:00"); + value2.setFireTime("2021-09-07 12:11:00"); + value2.setPartitionNum(100002); + WindowValue value3 = new WindowValue(); + value3.setStartTime("2021-09-07 11:10:00"); + value3.setEndTime("2021-09-07 11:20:00"); + value3.setFireTime("2021-09-07 11:25:00"); + value3.setPartitionNum(100003); + // + String prefix = "sorted_session_window_key"; + String queueId = "001"; + String groupByValue = "default"; + String localPrefix = prefix + queueId; + String sortKey1 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value1.getFireTime(), String.valueOf(value1.getPartitionNum()), groupByValue); + String sortKey2 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value2.getFireTime(), String.valueOf(value2.getPartitionNum()), groupByValue); + String sortKey3 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value3.getFireTime(), String.valueOf(value3.getPartitionNum()), groupByValue); + valueMap.put(sortKey1, value1); + valueMap.put(sortKey2, value2); + valueMap.put(sortKey3, value3); + storage.multiPut(valueMap); + // + WindowStorage.WindowBaseValueIterator iterator = storage.loadWindowInstanceSplitData(prefix, queueId, windowInstance.createWindowInstanceId(), null, WindowValue.class); + List valueList = new ArrayList<>(); + while (iterator.hasNext()) { + WindowValue value = iterator.next(); + valueList.add(value); + } + Assert.assertEquals(3, valueList.size()); + Assert.assertEquals("2021-09-07 11:25:00", valueList.get(1).getFireTime()); + // + List sortList = new ArrayList<>(valueMap.values()); + Collections.sort(sortList, Comparator.comparing(WindowValue::getStartTime)); + for (WindowValue value : sortList) { + System.out.println(value.getStartTime() + " " + value.getEndTime() + " " + value.getFireTime()); + } + // + WindowValue value4 = new WindowValue(); + value4.setStartTime("2021-09-07 11:10:00"); + value4.setEndTime("2021-09-07 11:21:00"); + value4.setFireTime("2021-09-07 11:25:00"); + value4.setPartitionNum(100003); + String sortKey4 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value4.getFireTime(), String.valueOf(value4.getPartitionNum()), groupByValue); + valueMap.put(sortKey4, value4); + storage.multiPut(valueMap); + iterator = storage.loadWindowInstanceSplitData(prefix, queueId, windowInstance.createWindowInstanceId(), null, WindowValue.class); + valueList = new ArrayList<>(); + while (iterator.hasNext()) { + WindowValue value = iterator.next(); + valueList.add(value); + } + for (WindowValue value : valueList) { + System.out.println(value.getStartTime() + " " + value.getEndTime() + " " + value.getFireTime() + " " + value.getPartitionNum()); + } + // + storage.removeKeys(valueMap.keySet()); + } + +} diff --git a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java index 22112095..445387b9 100644 --- a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java +++ b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java @@ -17,12 +17,19 @@ package org.apache.rocketmq.streams.window; import com.alibaba.fastjson.JSONObject; +import java.util.ArrayList; +import java.util.Date; import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.rocketmq.streams.common.context.Message; +import org.apache.rocketmq.streams.common.utils.CollectionUtil; import org.apache.rocketmq.streams.common.utils.DateUtil; import org.apache.rocketmq.streams.db.driver.orm.ORMUtil; +import org.apache.rocketmq.streams.window.model.FireMode; import org.apache.rocketmq.streams.window.model.WindowInstance; +import org.apache.rocketmq.streams.window.operator.AbstractWindow; import org.apache.rocketmq.streams.window.operator.impl.WindowOperator; +import org.junit.Assert; import org.junit.Test; import static junit.framework.TestCase.assertTrue; @@ -118,4 +125,164 @@ public void testWindowInstanceMode2() { assertTrue(windowInstance.getSplitId().equals("1")); assertTrue(windowInstance.getFireTime().compareTo(DateUtil.getCurrentTimeString()) > 0); } + + public List hit(String shardId, AbstractWindow window, Long occurTime, Long sysTime, + int timeUnitAdjust) { + List instanceList = new ArrayList<>(); + int slideInterval = window.getSlideInterval(); + int sizeInterval = window.getSizeInterval(); + if (slideInterval == 0) { + slideInterval = sizeInterval; + } + int maxLateness = window.getWaterMarkMinute(); + List windowBeginTimeList = DateUtil.getWindowBeginTime(occurTime, slideInterval * timeUnitAdjust * 1000, + sizeInterval * timeUnitAdjust * 1000); + for (Date begin : windowBeginTimeList) { + Date end = DateUtil.addDate(TimeUnit.SECONDS, begin, sizeInterval * timeUnitAdjust); + Date fire = DateUtil.addDate(TimeUnit.SECONDS, end, maxLateness * timeUnitAdjust); + Date actualFireDate = fire; + if (window.getFireMode() != 0) { + if (sysTime == null || sysTime - end.getTime() < 0) { + actualFireDate = end; + } else { + List currentWindowList = DateUtil.getWindowBeginTime( + sysTime, slideInterval * timeUnitAdjust * 1000, + sizeInterval * timeUnitAdjust * 1000); + if (!CollectionUtil.isEmpty(currentWindowList)) { + Date soonBegin = currentWindowList.get(currentWindowList.size() - 1); + Date soonEnd = DateUtil.addDate(TimeUnit.SECONDS, soonBegin, + sizeInterval * timeUnitAdjust); + Date soonFire = soonEnd; + Date recentFireTime = DateUtil.addDate(TimeUnit.SECONDS, soonFire, -(sizeInterval * timeUnitAdjust)); + Date stopTime = DateUtil.addDate(TimeUnit.SECONDS, end, maxLateness * timeUnitAdjust * 1000); + //窗口结束时间处于窗口中间位置 + if (recentFireTime.compareTo(stopTime) < 0 && stopTime.compareTo(soonFire) <= 0) { + actualFireDate = stopTime; + } else { + actualFireDate = soonFire; + } + } + if (actualFireDate.getTime() - end.getTime() - maxLateness * timeUnitAdjust * 1000 > 0) { + System.err.println("window instance out of date!!! " + DateUtil.format(begin) + "-" + DateUtil.format(end) + "---" + DateUtil.format(fire)); + break; + } + } + } else { + if (sysTime != null && sysTime - actualFireDate.getTime() > 0) { + System.err.println("window instance out of date!!! " + DateUtil.format(begin) + "-" + DateUtil.format(end) + "---" + DateUtil.format(fire)); + break; + } + } + String startTime = DateUtil.format(begin); + String endTime = DateUtil.format(end); + String fireTime = DateUtil.format(fire); + String actualFireTime = DateUtil.format(actualFireDate); + //keep the state util fire time when fire mode == 2 + WindowInstance instance = (FireMode.ACCUMULATED.equals(FireMode.valueOf(window.getFireMode()))) ? window.createWindowInstance(startTime, endTime, fireTime, shardId) : window.createWindowInstance(startTime, endTime, actualFireTime, shardId); +// instance.setActualFireTime(actualFireTime); + instanceList.add(instance); + } + return instanceList; + } + + @Test + @Deprecated + public void testHitWindowInstance() { + //NORMAL + WindowOperator window = new WindowOperator(); + window.setFireMode(0); + window.setTimeFieldName("time"); + window.setSlideInterval(5); + window.setSizeInterval(5); + window.setWaterMarkMinute(0); + window.setNameSpace("namespace_test"); + window.setConfigureName("window_name"); + + JSONObject msg = new JSONObject(); + String eventTime = "2021-08-27 18:03:00"; + msg.put("time", eventTime); + + String shardId = "000"; + String currentTime = "2021-08-27 18:04:00"; + Long occurTimeStamp = DateUtil.parseTime(eventTime).getTime(); + Long currentTimeStamp = DateUtil.parseTime(currentTime).getTime(); + + List hitList = hit(shardId, window, occurTimeStamp, currentTimeStamp, 60); + Assert.assertEquals(1, hitList.size()); + Assert.assertEquals("2021-08-27 18:00:00", hitList.get(0).getStartTime()); + Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getFireTime()); +// Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getActualFireTime()); + + String anotherEventTime = "2021-08-27 18:04:00"; + Long anotherEventTimeStamp = DateUtil.parseTime(anotherEventTime).getTime(); + hitList = hit(shardId, window, anotherEventTimeStamp, currentTimeStamp, 60); + Assert.assertEquals(1, hitList.size()); + Assert.assertEquals("2021-08-27 18:00:00", hitList.get(0).getStartTime()); + Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getFireTime()); +// Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getActualFireTime()); + + //PARTITIONED + window = new WindowOperator(); + window.setFireMode(1); + window.setTimeFieldName("time"); + window.setSlideInterval(5); + window.setSizeInterval(5); + window.setWaterMarkMinute(120); + window.setNameSpace("namespace_test"); + window.setConfigureName("window_name"); + + msg = new JSONObject(); + eventTime = "2021-08-27 17:03:00"; + msg.put("time", eventTime); + + currentTime = "2021-08-27 18:00:00"; + occurTimeStamp = DateUtil.parseTime(eventTime).getTime(); + currentTimeStamp = DateUtil.parseTime(currentTime).getTime(); + + hitList = hit(shardId, window, occurTimeStamp, currentTimeStamp, 60); + Assert.assertEquals(1, hitList.size()); + Assert.assertEquals("2021-08-27 17:00:00", hitList.get(0).getStartTime()); + Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getFireTime()); +// Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getActualFireTime()); + + anotherEventTime = "2021-08-27 17:04:00"; + anotherEventTimeStamp = DateUtil.parseTime(anotherEventTime).getTime(); + hitList = hit(shardId, window, anotherEventTimeStamp, currentTimeStamp, 60); + Assert.assertEquals(1, hitList.size()); + Assert.assertEquals("2021-08-27 17:00:00", hitList.get(0).getStartTime()); + Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getFireTime()); +// Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getActualFireTime()); + + //ACCUMULATED + window = new WindowOperator(); + window.setFireMode(1); + window.setTimeFieldName("time"); + window.setSlideInterval(5); + window.setSizeInterval(5); + window.setWaterMarkMinute(120); + window.setNameSpace("namespace_test"); + window.setConfigureName("window_name"); + + msg = new JSONObject(); + eventTime = "2021-08-27 17:03:00"; + msg.put("time", eventTime); + + currentTime = "2021-08-27 18:00:00"; + occurTimeStamp = DateUtil.parseTime(eventTime).getTime(); + currentTimeStamp = DateUtil.parseTime(currentTime).getTime(); + + hitList = hit(shardId, window, occurTimeStamp, currentTimeStamp, 60); + Assert.assertEquals(1, hitList.size()); + Assert.assertEquals("2021-08-27 17:00:00", hitList.get(0).getStartTime()); + Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getFireTime()); +// Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getActualFireTime()); + + anotherEventTime = "2021-08-27 17:04:00"; + anotherEventTimeStamp = DateUtil.parseTime(anotherEventTime).getTime(); + hitList = hit(shardId, window, anotherEventTimeStamp, currentTimeStamp, 60); + Assert.assertEquals(1, hitList.size()); + Assert.assertEquals("2021-08-27 17:00:00", hitList.get(0).getStartTime()); + Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getFireTime()); +// Assert.assertEquals("2021-08-27 18:05:00", hitList.get(0).getActualFireTime()); + } }