You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by du...@apache.org on 2021/08/05 12:54:02 UTC
[rocketmq-streams] 04/46: add rocketmq-streams,
rocketmq-channel-rocketmq, rocketmq-serviceloader module
This is an automated email from the ASF dual-hosted git repository.
duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-streams.git
commit 2564d592a288c8548a83662e65e5eb1d2ac96944
Author: 赤玄 <yu...@alibaba-inc.com>
AuthorDate: Mon Aug 2 11:58:20 2021 +0800
add rocketmq-streams, rocketmq-channel-rocketmq, rocketmq-serviceloader module
---
rocketmq-streams-channel-rocketmq/pom.xml | 87 +
.../rocketmq-streams-channel-rocketmq.iml | 74 +
.../rocketmq/streams/RocketMQChannelBuilder.java | 90 +
.../apache/rocketmq/streams/RocketMQOffset.java | 89 +
.../streams/queue/RocketMQMessageQueue.java | 107 +
.../apache/rocketmq/streams/sink/RocketMQSink.java | 318 +++
.../rocketmq/streams/source/RocketMQSource.java | 292 ++
.../rocketmq/streams/AbstractChannelTest.java | 58 +
.../rocketmq/streams/RocketMQChannelTest.java | 176 ++
....streams.common.channel.builder.IChannelBuilder | 1 +
.../rocketmq/streams/RocketMQChannelBuilder.class | Bin 0 -> 3971 bytes
.../apache/rocketmq/streams/RocketMQOffset.class | Bin 0 -> 3628 bytes
.../streams/queue/RocketMQMessageQueue.class | Bin 0 -> 3849 bytes
.../rocketmq/streams/sink/RocketMQSink$1.class | Bin 0 -> 2040 bytes
.../rocketmq/streams/sink/RocketMQSink.class | Bin 0 -> 10455 bytes
.../rocketmq/streams/source/RocketMQSource$1.class | Bin 0 -> 3170 bytes
.../rocketmq/streams/source/RocketMQSource.class | Bin 0 -> 12146 bytes
.../target/maven-archiver/pom.properties | 5 +
...ams-channel-rocketmq-2.0.0-SNAPSHOT-sources.jar | Bin 0 -> 13382 bytes
...tmq-streams-channel-rocketmq-2.0.0-SNAPSHOT.jar | Bin 0 -> 20715 bytes
rocketmq-streams-serviceloader/pom.xml | 24 +
.../rocketmq-streams-serviceloader.iml | 61 +
.../serviceloader/IServiceLoaderService.java | 31 +
.../serviceloader/ServiceLoaderComponent.java | 132 +
.../namefinder/IServiceNameGetter.java | 30 +
.../impl/AnnotationServiceNameGetter.java | 39 +
.../streams/serviceloader/utils/FileUtil.java | 73 +
.../component/ServiceLoaderComponent.properties | 1 +
.../serviceloader/ServiceLoaderComponentTest.java | 17 +
.../src/test/resources/log4j.xml | 20 +
...ams.serviceloader.namefinder.IServiceNameGetter | 1 +
.../component/ServiceLoaderComponent.properties | 1 +
.../serviceloader/IServiceLoaderService.class | Bin 0 -> 413 bytes
.../serviceloader/ServiceLoaderComponent.class | Bin 0 -> 6038 bytes
.../namefinder/IServiceNameGetter.class | Bin 0 -> 235 bytes
.../impl/AnnotationServiceNameGetter.class | Bin 0 -> 1138 bytes
.../streams/serviceloader/utils/FileUtil.class | Bin 0 -> 3352 bytes
.../target/maven-archiver/pom.properties | 5 +
...treams-serviceloader-2.0.0-SNAPSHOT-sources.jar | Bin 0 -> 8077 bytes
...cketmq-streams-serviceloader-2.0.0-SNAPSHOT.jar | Bin 0 -> 9883 bytes
.../target/test-classes/log4j.xml | 20 +
.../serviceloader/ServiceLoaderComponentTest.class | Bin 0 -> 1558 bytes
rocketmq-streams-window/pom.xml | 36 +
.../rocketmq-streams-window.iml | 82 +
.../streams/window/builder/WindowBuilder.java | 95 +
.../streams/window/model/FunctionExecutor.java | 51 +
.../rocketmq/streams/window/model/WindowCache.java | 149 +
.../streams/window/model/WindowInstance.java | 506 ++++
.../window/offset/IWindowMaxValueManager.java | 87 +
.../streams/window/offset/WindowMaxValue.java | 89 +
.../window/offset/WindowMaxValueManager.java | 247 ++
.../window/operator/AbstractShuffleWindow.java | 83 +
.../streams/window/operator/AbstractWindow.java | 781 ++++++
.../streams/window/operator/impl/OverWindow.java | 154 ++
.../window/operator/impl/SessionWindow.java | 248 ++
.../window/operator/impl/WindowOperator.java | 451 +++
.../streams/window/operator/join/DBOperator.java | 427 +++
.../streams/window/operator/join/JoinWindow.java | 553 ++++
.../streams/window/operator/join/Operator.java | 76 +
.../window/shuffle/AbstractSystemChannel.java | 321 +++
.../streams/window/shuffle/ShuffleChannel.java | 542 ++++
.../streams/window/source/WindowRireSource.java | 336 +++
.../streams/window/state/WindowBaseValue.java | 188 ++
.../streams/window/state/impl/JoinLeftState.java | 20 +
.../streams/window/state/impl/JoinRightState.java | 20 +
.../streams/window/state/impl/JoinState.java | 94 +
.../streams/window/state/impl/WindowValue.java | 548 ++++
.../window/storage/AbstractWindowStorage.java | 120 +
.../streams/window/storage/ICommonStorage.java | 50 +
.../streams/window/storage/IKeyGenerator.java | 29 +
.../window/storage/IShufflePartitionManager.java | 52 +
.../rocketmq/streams/window/storage/IStorage.java | 79 +
.../streams/window/storage/IWindowStorage.java | 68 +
.../window/storage/ShufflePartitionManager.java | 86 +
.../streams/window/storage/StorageManager.java | 307 +++
.../streams/window/storage/WindowStorage.java | 183 ++
.../streams/window/storage/db/DBStorage.java | 186 ++
.../streams/window/storage/file/FileStorage.java | 196 ++
.../window/storage/rocksdb/RocksdbStorage.java | 293 ++
.../src/main/resources/dipper.cs | 2892 ++++++++++++++++++++
.../src/main/resources/dipper.properties | 21 +
.../streams/configruation/model/Person.java | 84 +
.../streams/configurable/model/Person.java | 97 +
.../streams/window/WindowInstanceTest.java | 122 +
.../streams/window/WindowMaxValueTest.java | 35 +
.../rocketmq/streams/window/WindowValueTest.java | 36 +
.../src/test/resources/log4j.xml | 20 +
rocketmq-streams-window/target/classes/dipper.cs | 2892 ++++++++++++++++++++
.../target/classes/dipper.properties | 21 +
.../streams/window/builder/WindowBuilder.class | Bin 0 -> 3057 bytes
.../streams/window/model/FunctionExecutor.class | Bin 0 -> 1591 bytes
.../streams/window/model/WindowCache.class | Bin 0 -> 6328 bytes
.../streams/window/model/WindowInstance.class | Bin 0 -> 14044 bytes
.../window/offset/IWindowMaxValueManager.class | Bin 0 -> 1137 bytes
.../streams/window/offset/WindowMaxValue.class | Bin 0 -> 2113 bytes
.../window/offset/WindowMaxValueManager.class | Bin 0 -> 9412 bytes
.../window/operator/AbstractShuffleWindow.class | Bin 0 -> 4175 bytes
.../streams/window/operator/AbstractWindow$1.class | Bin 0 -> 1063 bytes
.../streams/window/operator/AbstractWindow.class | Bin 0 -> 29704 bytes
.../streams/window/operator/impl/OverWindow.class | Bin 0 -> 5387 bytes
.../window/operator/impl/SessionWindow.class | Bin 0 -> 13002 bytes
.../window/operator/impl/WindowOperator$1.class | Bin 0 -> 1944 bytes
.../impl/WindowOperator$WindowRowOperator.class | Bin 0 -> 3973 bytes
.../window/operator/impl/WindowOperator.class | Bin 0 -> 17416 bytes
.../streams/window/operator/join/DBOperator.class | Bin 0 -> 16903 bytes
.../window/operator/join/JoinWindow$1.class | Bin 0 -> 2931 bytes
.../window/operator/join/JoinWindow$2.class | Bin 0 -> 2870 bytes
.../streams/window/operator/join/JoinWindow.class | Bin 0 -> 21377 bytes
.../streams/window/operator/join/Operator.class | Bin 0 -> 2912 bytes
.../window/shuffle/AbstractSystemChannel.class | Bin 0 -> 11888 bytes
.../ShuffleChannel$ShuffleOutputDataSource.class | Bin 0 -> 5054 bytes
.../streams/window/shuffle/ShuffleChannel.class | Bin 0 -> 22485 bytes
.../window/source/WindowRireSource$1$1.class | Bin 0 -> 1524 bytes
.../streams/window/source/WindowRireSource$1.class | Bin 0 -> 2346 bytes
.../WindowRireSource$WindowInstanceCache$1$1.class | Bin 0 -> 1833 bytes
.../WindowRireSource$WindowInstanceCache$1.class | Bin 0 -> 2199 bytes
.../WindowRireSource$WindowInstanceCache.class | Bin 0 -> 1592 bytes
.../streams/window/source/WindowRireSource.class | Bin 0 -> 8081 bytes
.../streams/window/state/WindowBaseValue.class | Bin 0 -> 4083 bytes
.../streams/window/state/impl/JoinLeftState.class | Bin 0 -> 395 bytes
.../streams/window/state/impl/JoinRightState.class | Bin 0 -> 398 bytes
.../streams/window/state/impl/JoinState.class | Bin 0 -> 1943 bytes
.../streams/window/state/impl/WindowValue$1.class | Bin 0 -> 930 bytes
.../streams/window/state/impl/WindowValue.class | Bin 0 -> 23728 bytes
.../window/storage/AbstractWindowStorage$1.class | Bin 0 -> 2596 bytes
.../window/storage/AbstractWindowStorage.class | Bin 0 -> 6724 bytes
.../streams/window/storage/ICommonStorage.class | Bin 0 -> 968 bytes
.../streams/window/storage/IKeyGenerator.class | Bin 0 -> 193 bytes
.../window/storage/IShufflePartitionManager.class | Bin 0 -> 642 bytes
.../rocketmq/streams/window/storage/IStorage.class | Bin 0 -> 2875 bytes
.../streams/window/storage/IWindowStorage.class | Bin 0 -> 2282 bytes
.../window/storage/ShufflePartitionManager.class | Bin 0 -> 2851 bytes
.../streams/window/storage/StorageManager.class | Bin 0 -> 353 bytes
.../streams/window/storage/WindowStorage$1.class | Bin 0 -> 1402 bytes
.../WindowStorage$WindowBaseValueIterator.class | Bin 0 -> 980 bytes
.../streams/window/storage/WindowStorage.class | Bin 0 -> 7364 bytes
.../streams/window/storage/db/DBStorage$1.class | Bin 0 -> 1523 bytes
.../window/storage/db/DBStorage$DBIterator.class | Bin 0 -> 4299 bytes
.../streams/window/storage/db/DBStorage.class | Bin 0 -> 7161 bytes
.../window/storage/file/FileStorage$1.class | Bin 0 -> 2620 bytes
.../streams/window/storage/file/FileStorage.class | Bin 0 -> 9198 bytes
.../rocksdb/RocksdbStorage$LocalIterator.class | Bin 0 -> 3346 bytes
.../window/storage/rocksdb/RocksdbStorage.class | Bin 0 -> 11737 bytes
.../target/maven-archiver/pom.properties | 5 +
...ketmq-streams-window-2.0.0-SNAPSHOT-sources.jar | Bin 0 -> 357527 bytes
.../rocketmq-streams-window-2.0.0-SNAPSHOT.jar | Bin 0 -> 403392 bytes
.../target/test-classes/log4j.xml | 20 +
147 files changed, 15830 insertions(+)
diff --git a/rocketmq-streams-channel-rocketmq/pom.xml b/rocketmq-streams-channel-rocketmq/pom.xml
new file mode 100644
index 0000000..835322c
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/pom.xml
@@ -0,0 +1,87 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <parent>
+ <artifactId>rocketmq-streams</artifactId>
+ <groupId>org.apache.rocketmq</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ </parent>
+ <modelVersion>4.0.0</modelVersion>
+
+ <artifactId>rocketmq-streams-channel-rocketmq</artifactId>
+ <name>ROCKETMQ STREAMS :: channel-rocketmq</name>
+ <packaging>jar</packaging>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.rocketmq</groupId>
+ <artifactId>rocketmq-streams-commons</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.rocketmq</groupId>
+ <artifactId>rocketmq-streams-serviceloader</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.rocketmq</groupId>
+ <artifactId>rocketmq-client</artifactId>
+ <exclusions>
+ <exclusion>
+ <groupId>ch.qos.logback</groupId>
+ <artifactId>logback-classic</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>ch.qos.logback</groupId>
+ <artifactId>logback-core</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.rocketmq</groupId>
+ <artifactId>rocketmq-acl</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ <version>1.7.7</version>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>jcl-over-slf4j</artifactId>
+ <version>1.7.7</version>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ <version>1.7.7</version>
+ </dependency>
+ <dependency>
+ <groupId>log4j</groupId>
+ <artifactId>log4j</artifactId>
+ <version>1.2.17</version>
+ </dependency>
+ </dependencies>
+
+</project>
\ No newline at end of file
diff --git a/rocketmq-streams-channel-rocketmq/rocketmq-streams-channel-rocketmq.iml b/rocketmq-streams-channel-rocketmq/rocketmq-streams-channel-rocketmq.iml
new file mode 100644
index 0000000..d0b5aa3
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/rocketmq-streams-channel-rocketmq.iml
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<module org.jetbrains.idea.maven.project.MavenProjectsManager.isMavenModule="true" type="JAVA_MODULE" version="4">
+ <component name="NewModuleRootManager" LANGUAGE_LEVEL="JDK_1_8">
+ <output url="file://$MODULE_DIR$/target/classes" />
+ <output-test url="file://$MODULE_DIR$/target/test-classes" />
+ <content url="file://$MODULE_DIR$">
+ <sourceFolder url="file://$MODULE_DIR$/src/main/java" isTestSource="false" />
+ <sourceFolder url="file://$MODULE_DIR$/src/test/java" isTestSource="true" />
+ <excludeFolder url="file://$MODULE_DIR$/target" />
+ </content>
+ <orderEntry type="inheritedJdk" />
+ <orderEntry type="sourceFolder" forTests="false" />
+ <orderEntry type="module" module-name="rocketmq-streams-commons" />
+ <orderEntry type="library" name="Maven: org.apache.commons:commons-lang3:3.11" level="project" />
+ <orderEntry type="library" name="Maven: com.alibaba:fastjson:1.2.27" level="project" />
+ <orderEntry type="library" name="Maven: commons-logging:commons-logging:1.1" level="project" />
+ <orderEntry type="library" name="Maven: logkit:logkit:1.0.1" level="project" />
+ <orderEntry type="library" name="Maven: avalon-framework:avalon-framework:4.1.3" level="project" />
+ <orderEntry type="library" name="Maven: javax.servlet:servlet-api:2.3" level="project" />
+ <orderEntry type="library" name="Maven: commons-io:commons-io:2.5" level="project" />
+ <orderEntry type="library" name="Maven: com.google.code.gson:gson:2.8.5" level="project" />
+ <orderEntry type="library" name="Maven: com.google.auto.service:auto-service:1.0-rc5" level="project" />
+ <orderEntry type="library" name="Maven: com.google.auto.service:auto-service-annotations:1.0-rc5" level="project" />
+ <orderEntry type="library" name="Maven: com.google.auto:auto-common:0.10" level="project" />
+ <orderEntry type="library" name="Maven: com.google.guava:guava:25.1-jre" level="project" />
+ <orderEntry type="library" name="Maven: com.google.code.findbugs:jsr305:3.0.2" level="project" />
+ <orderEntry type="library" name="Maven: org.checkerframework:checker-qual:2.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.google.errorprone:error_prone_annotations:2.1.3" level="project" />
+ <orderEntry type="library" name="Maven: com.google.j2objc:j2objc-annotations:1.1" level="project" />
+ <orderEntry type="library" name="Maven: org.codehaus.mojo:animal-sniffer-annotations:1.14" level="project" />
+ <orderEntry type="library" name="Maven: com.lmax:disruptor:3.2.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:hyperscan:5.4.0-2.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:linux-x86_64:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:windows-x86_64:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:macosx-x86_64:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp-platform:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-arm:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-arm64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-x86:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:ios-arm64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:ios-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-armhf:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-arm64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-ppc64le:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-x86:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:macosx-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:windows-x86:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:windows-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: net.java.dev.jna:platform:3.5.2" level="project" />
+ <orderEntry type="library" name="Maven: net.java.dev.jna:jna:3.5.2" level="project" />
+ <orderEntry type="module" module-name="rocketmq-streams-serviceloader" />
+ <orderEntry type="library" name="Maven: org.apache.rocketmq:rocketmq-client:4.5.2" level="project" />
+ <orderEntry type="library" name="Maven: org.apache.rocketmq:rocketmq-common:4.5.2" level="project" />
+ <orderEntry type="library" name="Maven: org.apache.rocketmq:rocketmq-acl:4.5.2" level="project" />
+ <orderEntry type="library" name="Maven: org.apache.rocketmq:rocketmq-remoting:4.5.2" level="project" />
+ <orderEntry type="library" name="Maven: io.netty:netty-all:4.0.42.Final" level="project" />
+ <orderEntry type="library" name="Maven: io.netty:netty-tcnative-boringssl-static:1.1.33.Fork26" level="project" />
+ <orderEntry type="library" name="Maven: org.apache.rocketmq:rocketmq-logging:4.5.2" level="project" />
+ <orderEntry type="library" name="Maven: org.apache.rocketmq:rocketmq-srvutil:4.5.2" level="project" />
+ <orderEntry type="library" name="Maven: commons-cli:commons-cli:1.2" level="project" />
+ <orderEntry type="library" name="Maven: org.yaml:snakeyaml:1.19" level="project" />
+ <orderEntry type="library" name="Maven: commons-codec:commons-codec:1.9" level="project" />
+ <orderEntry type="library" scope="TEST" name="Maven: junit:junit:4.12" level="project" />
+ <orderEntry type="library" scope="TEST" name="Maven: org.hamcrest:hamcrest-core:1.3" level="project" />
+ <orderEntry type="library" name="Maven: org.slf4j:slf4j-api:1.7.7" level="project" />
+ <orderEntry type="library" name="Maven: org.slf4j:jcl-over-slf4j:1.7.7" level="project" />
+ <orderEntry type="library" name="Maven: org.slf4j:slf4j-log4j12:1.7.7" level="project" />
+ <orderEntry type="library" name="Maven: log4j:log4j:1.2.17" level="project" />
+ </component>
+</module>
\ No newline at end of file
diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQChannelBuilder.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQChannelBuilder.java
new file mode 100644
index 0000000..e83b1dc
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQChannelBuilder.java
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import com.alibaba.fastjson.JSONObject;
+import com.google.auto.service.AutoService;
+import org.apache.rocketmq.streams.common.channel.builder.AbstractSupportShuffleChannelBuilder;
+import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder;
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.metadata.MetaData;
+import org.apache.rocketmq.streams.common.model.ServiceName;
+import org.apache.rocketmq.streams.common.utils.ConfigurableUtil;
+import org.apache.rocketmq.streams.sink.RocketMQSink;
+import org.apache.rocketmq.streams.source.RocketMQSource;
+
+import java.util.Properties;
+
+@AutoService(IChannelBuilder.class)
+@ServiceName(value = RocketMQChannelBuilder.TYPE, aliasName = "RocketMQSource")
+public class RocketMQChannelBuilder extends AbstractSupportShuffleChannelBuilder {
+ public static final String TYPE = "rocketmq";
+
+ @Override
+ public ISource createSource(String namespace, String name, Properties properties, MetaData metaData) {
+
+ RocketMQSource rocketMQSource = (RocketMQSource) ConfigurableUtil.create(RocketMQSource.class.getName(),namespace,name,createFormatProperty(properties),null);
+ return rocketMQSource;
+ }
+
+ protected JSONObject createFormatProperty(Properties properties){
+ JSONObject formatProperties=new JSONObject();
+ for(Object object:properties.keySet()){
+ String key=(String)object;
+ if ("type".equals(key)) {
+ continue;
+ }
+ formatProperties.put(key,properties.getProperty(key));
+ }
+ IChannelBuilder.formatPropertiesName(formatProperties,properties,"topic","topic");
+ IChannelBuilder.formatPropertiesName(formatProperties,properties,"tags","tag");
+ IChannelBuilder.formatPropertiesName(formatProperties,properties,"maxThread","thread.max.count");
+ IChannelBuilder.formatPropertiesName(formatProperties,properties,"pullIntervalMs","pullIntervalMs");
+ IChannelBuilder.formatPropertiesName(formatProperties,properties,"offsetTime","offsetTime");
+ IChannelBuilder.formatPropertiesName(formatProperties,properties,"namesrvAddr","namesrvAddr");
+ if (properties.getProperty("group") != null) {
+ String group = properties.getProperty("group");
+ if (group.startsWith("GID_")) {
+ formatProperties.put("groupName", group);
+ } else {
+ formatProperties.put("groupName", "GID_" + group);
+ }
+ }
+
+
+ return formatProperties;
+ }
+
+
+ @Override
+ public String getType() {
+ return TYPE;
+ }
+
+ @Override
+ public ISink createSink(String namespace, String name, Properties properties, MetaData metaData) {
+ RocketMQSink rocketMQSink = (RocketMQSink) ConfigurableUtil.create(RocketMQSink.class.getName(),namespace,name,createFormatProperty(properties),null);
+ return rocketMQSink;
+ }
+
+ @Override
+ public ISink createBySource(ISource piplineSoure) {
+ return null;
+ }
+}
diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQOffset.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQOffset.java
new file mode 100644
index 0000000..ef0c322
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQOffset.java
@@ -0,0 +1,89 @@
+/*
+ * 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;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.rocketmq.client.consumer.store.OffsetStore;
+import org.apache.rocketmq.client.consumer.store.ReadOffsetType;
+import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.remoting.exception.RemotingException;
+import org.apache.rocketmq.streams.common.channel.source.AbstractSupportOffsetResetSource;
+import org.apache.rocketmq.streams.queue.RocketMQMessageQueue;
+
+public class RocketMQOffset implements OffsetStore {
+ protected OffsetStore offsetStore;
+ protected AbstractSupportOffsetResetSource source;
+ public RocketMQOffset(OffsetStore offsetStore, AbstractSupportOffsetResetSource source){
+ this.offsetStore=offsetStore;
+ this.source=source;
+ }
+ @Override
+ public void load() throws MQClientException {
+ offsetStore.load();
+ }
+
+ @Override
+ public void updateOffset(MessageQueue mq, long offset, boolean increaseOnly) {
+ offsetStore.updateOffset(mq,offset,increaseOnly);
+ }
+
+ @Override
+ public long readOffset(MessageQueue mq, ReadOffsetType type) {
+ return offsetStore.readOffset(mq,type);
+ }
+
+ @Override
+ public void persistAll(Set<MessageQueue> mqs) {
+ Set<String> queueIds=new HashSet<>();
+ for(MessageQueue mq:mqs){
+ queueIds.add(new RocketMQMessageQueue(mq).getQueueId());
+ }
+ source.sendCheckpoint(queueIds);
+ offsetStore.persistAll(mqs);
+ }
+
+ @Override
+ public void persist(MessageQueue mq) {
+ source.sendCheckpoint(new RocketMQMessageQueue(mq).getQueueId());
+ offsetStore.persist(mq);
+ }
+
+ @Override
+ public void removeOffset(MessageQueue mq) {
+ Set<String> splitIds = new HashSet<>();
+ splitIds.add(new RocketMQMessageQueue(mq).getQueueId());
+ source.removeSplit(splitIds);
+ offsetStore.removeOffset(mq);
+ offsetStore.removeOffset(mq);
+ }
+
+ @Override
+ public Map<MessageQueue, Long> cloneOffsetTable(String topic) {
+ return offsetStore.cloneOffsetTable(topic);
+ }
+
+ @Override
+ public void updateConsumeOffsetToBroker(MessageQueue mq, long offset, boolean isOneway)
+ throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
+ offsetStore.updateOffset(mq,offset,isOneway);
+ }
+}
diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/queue/RocketMQMessageQueue.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/queue/RocketMQMessageQueue.java
new file mode 100644
index 0000000..46f0ef9
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/queue/RocketMQMessageQueue.java
@@ -0,0 +1,107 @@
+package org.apache.rocketmq.streams.queue;
+
+import com.alibaba.fastjson.JSONObject;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
+import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
+
+public class RocketMQMessageQueue extends BasedConfigurable implements ISplit<RocketMQMessageQueue, MessageQueue> {
+ protected transient MessageQueue queue;
+ protected String brokeName;
+ protected String topic;
+ protected int mqQueueId;
+
+
+ @Override
+ protected void getJsonObject(JSONObject jsonObject) {
+ super.getJsonObject(jsonObject);
+ queue=new MessageQueue(topic,brokeName,mqQueueId);
+ }
+
+
+ public RocketMQMessageQueue(MessageQueue queue) {
+ this.queue = queue;
+ this.brokeName=queue.getBrokerName();
+ this.topic=queue.getTopic();
+ this.mqQueueId=queue.getQueueId();
+ }
+
+ public RocketMQMessageQueue() {
+
+ }
+
+ @Override
+ public MessageQueue getQueue() {
+ return queue;
+ }
+
+ @Override
+ public int compareTo(RocketMQMessageQueue o) {
+ return queue.compareTo(o.queue);
+ }
+
+
+
+ @Override
+ public String getQueueId() {
+ return getQueueId(this.queue);
+ }
+
+ @Override
+ public String getPlusQueueId() {
+ return MapKeyUtil.createKeyBySign("_",queue.getTopic(),queue.getBrokerName(),getSplitNumerStr(queue.getQueueId()+1)+"");
+ }
+
+ public static String getQueueId(MessageQueue queue){
+
+ String[] topic = queue.getTopic().split("%");
+ if (topic.length > 1) {
+ return MapKeyUtil.createKeyBySign("_",topic[1],queue.getBrokerName(),getSplitNumerStr(queue.getQueueId())+"");
+ }
+ return MapKeyUtil.createKeyBySign("_",queue.getTopic(),queue.getBrokerName(),getSplitNumerStr(queue.getQueueId())+"");
+ }
+ /**
+ * 获取分片的字符串格式,需要3位对齐
+ * @param splitNumer
+ * @return
+ */
+ private static String getSplitNumerStr(int splitNumer){
+ int len=(splitNumer+"").length();
+ if(len==3){
+ return splitNumer+"";
+ }
+ String splitNumerStr=splitNumer+"";
+ while (len<3){
+ splitNumerStr="0"+splitNumerStr;
+ len=splitNumerStr.length();
+ }
+ return splitNumerStr;
+ }
+
+ public String getBrokeName() {
+ return brokeName;
+ }
+
+ public void setBrokeName(String brokeName) {
+ this.brokeName = brokeName;
+ }
+
+ public String getTopic() {
+ return topic;
+ }
+
+ public void setTopic(String topic) {
+ this.topic = topic;
+ }
+
+ public int getMqQueueId() {
+ return mqQueueId;
+ }
+
+ public void setMqQueueId(int mqQueueId) {
+ this.mqQueueId = mqQueueId;
+ }
+
+
+}
diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java
new file mode 100644
index 0000000..8caf246
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java
@@ -0,0 +1,318 @@
+/*
+ * 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.sink;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.SendCallback;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+import org.apache.rocketmq.streams.queue.RocketMQMessageQueue;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class RocketMQSink extends AbstractSupportShuffleSink {
+
+ protected static final Log LOG = LogFactory.getLog(RocketMQSink.class);
+ @ENVDependence
+ protected String tags = "*";
+
+ protected String topic;
+ protected String groupName;
+
+ private transient List<DefaultMQPushConsumer> consumers=new ArrayList<>();
+ protected transient DefaultMQProducer producer;
+
+ protected Long pullIntervalMs;
+ protected String namesrvAddr;
+
+
+ public RocketMQSink(){}
+
+
+ @Override
+ protected boolean initConfigurable() {
+ super.initConfigurable();
+ return true;
+ }
+
+ protected transient AtomicBoolean isProcessing=new AtomicBoolean(false);
+ @Override
+ protected boolean batchInsert(List<IMessage> messages) {
+ if (messages == null) {
+ return true;
+ }
+ List<IMessage> needOrderProducer=new ArrayList<>();
+ for (int i=0;i<messages.size();i++) {
+ IMessage message =messages.get(i);
+ if(getSPlit(message)!=null){
+ if(i==0){
+ needOrderProducer=messages;
+ break;
+ }
+ needOrderProducer.add(message);
+ continue;
+ }
+ AtomicInteger msgFinishCount=new AtomicInteger(1);
+ putMessage2Mq(message,msgFinishCount,messages.size());
+ }
+ if(needOrderProducer.size()==0){
+ return true;
+ }
+ boolean success= isProcessing.compareAndSet(false,true);
+ if(!success){
+ while (isProcessing.get()){
+ synchronized (this){
+ try {
+ this.wait();
+ success= isProcessing.compareAndSet(false,true);
+ if(success){
+ break;
+ }
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+
+ }
+ }
+ }
+ AtomicInteger msgFinishCount=new AtomicInteger(0);
+ for(IMessage message:needOrderProducer){
+ putMessage2Mq(message,msgFinishCount,needOrderProducer.size());
+ }
+
+
+ return true;
+ }
+
+ protected boolean putMessage2Mq(IMessage fieldName2Value,AtomicInteger msgFinishCount,int allMsgSize) {
+ MessageQueue targetQueue = null;
+ ISplit<RocketMQMessageQueue,MessageQueue> channelQueue=getSPlit(fieldName2Value);
+
+ if (channelQueue!= null) {
+ targetQueue = channelQueue.getQueue();
+ }
+ sendMessage(fieldName2Value.getMessageValue().toString(), null, this.tags, targetQueue,msgFinishCount,allMsgSize);
+ return true;
+ }
+
+ /**
+ * 发送metaq消息
+ * @param content 消息内容
+ * @param key 消息的Key字段是为了唯一标识消息的,方便运维排查问题。如果不设置Key,则无法定位消息丢失原因。
+ * @param targetQueue
+ */
+ protected void sendMessage(String content, String key, String tags, MessageQueue targetQueue,AtomicInteger msgFinishCount,int allMsgSize) {
+ try {
+ if (StringUtil.isEmpty(topic)) {
+ if (LOG.isErrorEnabled()) {
+ LOG.error("topic is blank:" + content);
+ }
+ return;
+ }
+ initProducer();
+ final RocketMQSink rocketMQSink=this;
+ Message msg = new Message(topic, tags, key, content.getBytes("UTF-8"));
+ if (targetQueue != null) {
+ producer.send(msg, targetQueue,new SendCallback() {
+ @Override
+ public void onSuccess(SendResult sendResult) {
+ int finishCount=msgFinishCount.incrementAndGet();
+ if(finishCount==allMsgSize){
+ synchronized (rocketMQSink){
+ isProcessing.set(false);
+ rocketMQSink.notifyAll();;
+ }
+ }
+ }
+
+ @Override
+ public void onException(Throwable e) {
+ int finishCount=msgFinishCount.incrementAndGet();
+ if(finishCount==allMsgSize){
+ synchronized (rocketMQSink){
+ isProcessing.set(false);
+ rocketMQSink.notifyAll();;
+ }
+ }
+ }
+ });
+ } else {
+ producer.sendOneway(msg );
+ }
+ } catch (Exception e) {
+ LOG.error("send message error:" + content, e);
+ }
+ }
+
+ protected void initProducer() {
+ if(producer==null){
+ synchronized (this){
+ if(producer==null){
+ destroy();
+ producer = new DefaultMQProducer(groupName + "producer", true, null);
+ try {
+ if (this.namesrvAddr != null && !"".equalsIgnoreCase(this.namesrvAddr)) {
+ producer.setNamesrvAddr(this.namesrvAddr);
+ }
+ producer.start();
+ } catch (Exception e) {
+ setInitSuccess(false);
+ throw new RuntimeException("创建队列失败," + topic + ",msg=" + e.getMessage(), e);
+ }
+ }
+ }
+
+ }
+ }
+
+ public void destroyProduce() {
+ if (producer != null) {
+ try {
+ producer.shutdown();
+ producer=null;
+ } catch (Throwable t) {
+ if (LOG.isWarnEnabled()) {
+ LOG.warn(t.getMessage(), t);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void destroy() {
+ super.destroy();
+ destroyProduce();
+ }
+
+ @Override
+ public String getShuffleTopicFieldName() {
+ return "topic";
+ }
+
+ @Override
+ protected void createTopicIfNotExist(int splitNum) {
+
+ }
+
+ @Override
+ public List<ISplit> getSplitList() {
+ initProducer();
+ List<ISplit> messageQueues=new ArrayList<>();
+ try {
+
+ if (messageQueues == null || messageQueues.size() == 0) {
+ List<MessageQueue> metaqQueueSet = producer.fetchPublishMessageQueues(topic);
+ List<ISplit> queueList = new ArrayList<>();
+ for (MessageQueue queue : metaqQueueSet) {
+ RocketMQMessageQueue rocketMQMessageQueue = new RocketMQMessageQueue(queue);
+ queueList.add(rocketMQMessageQueue);
+
+ }
+ Collections.sort(queueList);
+ messageQueues = queueList;
+ }
+ }catch (Exception e){
+ throw new RuntimeException(e);
+ }
+
+ return messageQueues;
+ }
+
+ @Override
+ public int getSplitNum() {
+ List<ISplit> splits=getSplitList();
+ if(splits==null||splits.size()==0){
+ return 0;
+ }
+ Set<Integer> splitNames=new HashSet<>();
+ for(ISplit split:splits){
+ MessageQueue messageQueue= (MessageQueue)split.getQueue();
+ splitNames.add(messageQueue.getQueueId());
+ }
+ return splitNames.size();
+ }
+
+
+ public String getTags() {
+ return tags;
+ }
+
+ public void setTags(String tags) {
+ this.tags = tags;
+ }
+
+ public String getTopic() {
+ return topic;
+ }
+
+ public void setTopic(String topic) {
+ this.topic = topic;
+ }
+
+ public String getGroupName() {
+ return groupName;
+ }
+
+ public void setGroupName(String groupName) {
+ this.groupName = groupName;
+ }
+
+ public List<DefaultMQPushConsumer> getConsumers() {
+ return consumers;
+ }
+
+ public void setConsumers(List<DefaultMQPushConsumer> consumers) {
+ this.consumers = consumers;
+ }
+
+ public DefaultMQProducer getProducer() {
+ return producer;
+ }
+
+ public void setProducer(DefaultMQProducer producer) {
+ this.producer = producer;
+ }
+
+ public Long getPullIntervalMs() {
+ return pullIntervalMs;
+ }
+
+ public void setPullIntervalMs(Long pullIntervalMs) {
+ this.pullIntervalMs = pullIntervalMs;
+ }
+
+ public String getNamesrvAddr() {
+ return namesrvAddr;
+ }
+
+ public void setNamesrvAddr(String namesrvAddr) {
+ this.namesrvAddr = namesrvAddr;
+ }
+
+}
diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/RocketMQSource.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/RocketMQSource.java
new file mode 100644
index 0000000..c2af211
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/RocketMQSource.java
@@ -0,0 +1,292 @@
+/*
+ * 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.source;
+
+import com.alibaba.fastjson.JSONObject;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.client.AccessChannel;
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus;
+import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly;
+import org.apache.rocketmq.client.consumer.store.RemoteBrokerOffsetStore;
+import org.apache.rocketmq.client.impl.MQClientManager;
+import org.apache.rocketmq.client.impl.consumer.DefaultMQPushConsumerImpl;
+import org.apache.rocketmq.client.impl.consumer.ProcessQueue;
+import org.apache.rocketmq.client.impl.consumer.RebalancePushImpl;
+import org.apache.rocketmq.client.impl.factory.MQClientInstance;
+import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.common.protocol.NamespaceUtil;
+import org.apache.rocketmq.common.protocol.heartbeat.MessageModel;
+import org.apache.rocketmq.streams.common.channel.source.AbstractSupportOffsetResetSource;
+import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence;
+import org.apache.rocketmq.streams.common.utils.ReflectUtil;
+import org.apache.rocketmq.streams.RocketMQOffset;
+import org.apache.rocketmq.streams.queue.RocketMQMessageQueue;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class RocketMQSource extends AbstractSupportOffsetResetSource {
+
+ protected static final Log LOG = LogFactory.getLog(RocketMQSource.class);
+ @ENVDependence
+ protected String tags = "*";
+
+ private transient List<DefaultMQPushConsumer> consumers = new ArrayList<>();
+
+ protected Long pullIntervalMs;
+
+ /**
+ * 消息队列命名空间接入点
+ */
+ protected String namesrvAddr;
+
+ protected transient ConsumeFromWhere consumeFromWhere;//默认从哪里消费,不会被持久化。不设置默认从尾部消费
+ protected transient String consumerOffset;//从哪里开始消费
+
+ public RocketMQSource() {}
+
+ public RocketMQSource(String topic, String tags, String groupName, String endpoint,
+ String namesrvAddr, String accessKey, String secretKey, String instanceId) {
+ this.topic = topic;
+ this.tags = tags;
+ this.groupName = groupName;
+ this.namesrvAddr = namesrvAddr;
+ }
+
+ @Override
+ protected boolean initConfigurable() {
+ return super.initConfigurable();
+ }
+
+ @Override
+ protected boolean startSource() {
+ try {
+ destroyConsumer();
+ consumers.add(startConsumer());
+ return true;
+ } catch (Exception e) {
+ setInitSuccess(false);
+ e.printStackTrace();
+ throw new RuntimeException("start rocketmq channel error " + topic, e);
+ }
+ }
+
+ protected DefaultMQPushConsumer startConsumer() {
+ try {
+ DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(groupName);
+ if (pullIntervalMs != null) {
+ consumer.setPullInterval(pullIntervalMs);
+ }
+ // consumer.setConsumeThreadMax(maxThread);
+ // consumer.setConsumeThreadMin(maxThread);
+
+ consumer.setPersistConsumerOffsetInterval((int)this.checkpointTime);
+ consumer.setConsumeMessageBatchMaxSize(maxFetchLogGroupSize);
+ consumer.setAccessChannel(AccessChannel.CLOUD);
+ consumer.setNamesrvAddr(this.namesrvAddr);
+ if (consumeFromWhere != null) {
+ consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_TIMESTAMP);
+ if (consumerOffset != null) {
+ consumer.setConsumeTimestamp(consumerOffset);
+ }
+ }
+
+ consumer.subscribe(topic, tags);
+ consumer.registerMessageListener((MessageListenerOrderly)(msgs, context) -> {
+ try {
+ int i = 0;
+ for (MessageExt msg : msgs) {
+ String data = new String(msg.getBody(), CHARSET);
+ JSONObject jsonObject = create(data);
+ String queueId = RocketMQMessageQueue.getQueueId(context.getMessageQueue());
+ String offset = msg.getQueueOffset() + "";
+ org.apache.rocketmq.streams.common.context.Message message = createMessage(jsonObject, queueId, offset, false);
+ message.getHeader().setOffsetIsLong(true);
+ // message.getHeader().setQueueId(RocketMQMessageQueue.getQueueId(context.getMessageQueue()));
+ // message.getHeader().setOffset(String.valueOf(msg.getQueueOffset()));
+ // message.getHeader().setMessageQueue(new RocketMQMessageQueue(context.getMessageQueue()));
+ if (i == msgs.size() - 1) {
+ message.getHeader().setNeedFlush(true);
+ }
+ executeMessage(message);
+ i++;
+ }
+ } catch (Exception e) {
+ LOG.error("消费metaq报错:" + e, e);
+ }
+
+ return ConsumeOrderlyStatus.SUCCESS;// 返回消费成功
+ });
+
+ setOffsetStore(consumer);
+ addRebalanceCallback(consumer);
+ consumer.start();
+
+ return consumer;
+ } catch (Exception e) {
+ setInitSuccess(false);
+ e.printStackTrace();
+ throw new RuntimeException("start metaq channel error " + topic, e);
+ }
+ }
+
+ /**
+ * 设置offset存储,包装原有的RemoteBrokerOffsetStore,在保存offset前发送系统消息
+ *
+ * @param consumer
+ */
+ protected void setOffsetStore(DefaultMQPushConsumer consumer) {
+ DefaultMQPushConsumerImpl defaultMQPushConsumer = consumer.getDefaultMQPushConsumerImpl();
+ if (consumer.getMessageModel() == MessageModel.CLUSTERING) {
+ consumer.changeInstanceNameToPID();
+ }
+ MQClientInstance mQClientFactory = MQClientManager.getInstance().getAndCreateMQClientInstance(defaultMQPushConsumer.getDefaultMQPushConsumer());
+ RemoteBrokerOffsetStore offsetStore = new RemoteBrokerOffsetStore(mQClientFactory, NamespaceUtil.wrapNamespace(consumer.getNamespace(), consumer.getConsumerGroup()));
+ consumer.setOffsetStore(new RocketMQOffset(offsetStore, this));//每个一分钟运行一次
+ }
+
+ @Override
+ protected boolean isNotDataSplit(String queueId) {
+ return queueId.toUpperCase().startsWith("%RETRY%");
+ }
+
+ /**
+ * 分片发生变化时,回调系统函数,发送系统消息,告知各个组件
+ *
+ * @param consumer
+ */
+ protected void addRebalanceCallback(DefaultMQPushConsumer consumer) {
+ DefaultMQPushConsumerImpl defaultMQPushConsumerImpl = consumer.getDefaultMQPushConsumerImpl();
+ // DefaultMQPushConsumerImpl defaultMQPushConsumerImpl=metaPushConsumer.getDefaultMQPushConsumerImpl();
+ ReflectUtil.setBeanFieldValue(defaultMQPushConsumerImpl, "rebalanceImpl", new RebalancePushImpl(defaultMQPushConsumerImpl) {
+ @Override
+ public void messageQueueChanged(String topic, Set<MessageQueue> mqAll, Set<MessageQueue> mqDivided) {
+ Set<String> queueIds = new HashSet<>();
+ for (MessageQueue messageQueue : mqAll) {
+ if (!mqDivided.contains(messageQueue)) {
+ ProcessQueue pq = this.processQueueTable.remove(messageQueue);
+ if (pq != null) {
+ pq.setDropped(true);
+ log.info("doRebalance, {}, truncateMessageQueueNotMyTopic remove unnecessary mq, {}", consumerGroup, messageQueue);
+ }
+ queueIds.add(RocketMQMessageQueue.getQueueId(messageQueue));
+ }
+ }
+ Set<String> newQueueIds = new HashSet<>();
+ for (MessageQueue messageQueue : mqDivided) {
+ if (!mqAll.contains(messageQueue)) {
+ newQueueIds.add(RocketMQMessageQueue.getQueueId(messageQueue));
+ }
+ }
+ removeSplit(queueIds);
+ addNewSplit(newQueueIds);
+
+ }
+ });
+ }
+
+ @Override
+ public boolean supportNewSplitFind() {
+ return true;
+ }
+
+ @Override
+ public boolean supportRemoveSplitFind() {
+ return true;
+ }
+
+ @Override
+ public boolean supportOffsetRest() {
+ return false;
+ }
+
+ public void destroyConsumer() {
+ List<DefaultMQPushConsumer> oldConsumers = new ArrayList<>(consumers);
+ try {
+ for (DefaultMQPushConsumer consumer : oldConsumers) {
+ consumer.shutdown();
+ }
+
+ } catch (Throwable t) {
+ if (LOG.isWarnEnabled()) {
+ LOG.warn(t.getMessage(), t);
+ }
+ }
+
+ }
+
+ @Override
+ public void destroy() {
+ super.destroy();
+ destroyConsumer();
+ }
+
+ public String getTags() {
+ return tags;
+ }
+
+ public void setTags(String tags) {
+ this.tags = tags;
+ }
+
+ public List<DefaultMQPushConsumer> getConsumers() {
+ return consumers;
+ }
+
+ public void setConsumers(List<DefaultMQPushConsumer> consumers) {
+ this.consumers = consumers;
+ }
+
+ public Long getPullIntervalMs() {
+ return pullIntervalMs;
+ }
+
+ public void setPullIntervalMs(Long pullIntervalMs) {
+ this.pullIntervalMs = pullIntervalMs;
+ }
+
+ public String getNamesrvAddr() {
+ return namesrvAddr;
+ }
+
+ public void setNamesrvAddr(String namesrvAddr) {
+ this.namesrvAddr = namesrvAddr;
+ }
+
+ public ConsumeFromWhere getConsumeFromWhere() {
+ return consumeFromWhere;
+ }
+
+ public void setConsumeFromWhere(ConsumeFromWhere consumeFromWhere) {
+ this.consumeFromWhere = consumeFromWhere;
+ }
+
+ public String getConsumerOffset() {
+ return consumerOffset;
+ }
+
+ public void setConsumerOffset(String consumerOffset) {
+ this.consumerOffset = consumerOffset;
+ }
+}
\ No newline at end of file
diff --git a/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/AbstractChannelTest.java b/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/AbstractChannelTest.java
new file mode 100644
index 0000000..34fbf10
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/AbstractChannelTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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;
+
+import com.alibaba.fastjson.JSONObject;
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.junit.Test;
+
+public abstract class AbstractChannelTest {
+
+ @Test
+ public void testChannel() throws InterruptedException {
+ ISource channel = createSource();
+ channel.setGroupName("CID_YUNDUN_SOC_DIPPER_TEST");
+ channel.setMaxThread(1);
+ channel.start((message, context) -> {
+ //System.out.println(message.getMessageBody().getString(IChannel.OFFSET)+"-"+message.getMessageBody()
+ // .getString(IChannel.QUEUE_ID)+"-"+message.getMessageBody().getString(IChannel.IS_BATCH)+"-"+Thread
+ // .currentThread().getId());
+ System.out.println(message.getMessageBody());
+ return message;
+ });
+ while (true) {
+ Thread.sleep(1000L);
+ }
+ }
+
+ protected abstract ISource createSource();
+
+ protected abstract ISink createSink();
+
+ @Test
+ public void testOutput() {
+ ISink channel = createSink();
+ JSONObject message = new JSONObject();
+ message.put("name", "chris");
+ message.put("age", 18);
+ channel.batchAdd(new Message(message));
+ channel.flush();
+ }
+}
diff --git a/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/RocketMQChannelTest.java b/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/RocketMQChannelTest.java
new file mode 100644
index 0000000..e0a0503
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/src/test/java/org/apache/rocketmq/streams/RocketMQChannelTest.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.streams;
+
+import com.alibaba.fastjson.JSONObject;
+
+import java.util.*;
+
+import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink;
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.component.ComponentCreator;
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.apache.rocketmq.streams.common.interfaces.IStreamOperator;
+import org.apache.rocketmq.streams.common.utils.DateUtil;
+import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
+import org.junit.Test;
+
+public class RocketMQChannelTest extends AbstractChannelTest {
+ private static final String WINDOW_TASK_PROPERTY_KEY_PREFIX = "dipper.window.shuffle.rocketmq.dispatch.channel.";
+
+ @Test
+ public void testSource() throws InterruptedException {
+ ISource source = createSource();
+ source.start(new IStreamOperator() {
+ @Override
+ public Object doMessage(IMessage iMessage, AbstractContext context) {
+ System.out.println(iMessage.getMessageBody().toJSONString());
+ return null;
+ }
+ });
+
+ Thread.sleep(100000000l);
+ }
+
+ @Test
+ public void testSink() {
+ ISink sink = createSink();
+ sink.setBatchSize(1);
+ for (int i = 0; i < 1000; i++) {
+ sink.batchSave(createMsg());
+ }
+ }
+
+ @Test
+ public void testGetSplit() {
+ AbstractSupportShuffleSink sink = (AbstractSupportShuffleSink)createSink();
+ List<ISplit> splits = sink.getSplitList();
+ System.out.println(splits.size());
+ }
+
+ // @Test
+ // public void testCreateChannel() throws InterruptedException {
+ // RocketMQChannelBuilder builder = new RocketMQChannelBuilder();
+ // IChannel consumer = builder.createChannel(getWindowNameSpace(),getWindowName(), createChannelProperties(), null);
+ // IChannel producer = builder.createChannel(getWindowNameSpace(),getWindowName(), createChannelProperties(), null);
+ // consumer.start(new IMessageProcssor() {
+ // @Override
+ // public Object doMessage(IMessage message, AbstractContext context) {
+ // System.out.println(message.getMessageBody().toJSONString());
+ // return null;
+ // }
+ // });((AbstractBatchMessageChannel)producer).getQueueList();
+ //
+ // List<ChannelQueue> queueList = ((AbstractBatchMessageChannel)producer).getQueueList();
+ // for (int i=0; i<100000; i++) {
+ //
+ // producer.batchSave(createMsg(queueList.get(0)));
+ // }
+ // producer.flush();
+ //
+ // Thread.sleep(10000000l);
+ // }
+
+ public List<IMessage> createMsg() {
+ JSONObject obj = new JSONObject();
+ obj.put("test", "11111");
+ obj.put("time", DateUtil.format(new Date()));
+ IMessage message = new Message(obj);
+ List<IMessage> msgs = new ArrayList<>();
+ msgs.add(message);
+ return msgs;
+ }
+
+ protected Properties createChannelProperties() {
+ Properties properties = new Properties();
+ // properties = PropertiesUtils.getResourceProperties(this.getClass(), "com/aliyun/yundun/dipper/channel/adapter/dipper.properties");
+ Iterator<Map.Entry<Object, Object>> it = ComponentCreator.getProperties().entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry<Object, Object> entry = it.next();
+ String key = (String)entry.getKey();
+ String value = (String)entry.getValue();
+ if (key.startsWith(WINDOW_TASK_PROPERTY_KEY_PREFIX)) {
+ String channelKey = key.replace(WINDOW_TASK_PROPERTY_KEY_PREFIX, "");
+ properties.put(channelKey, value);
+ }
+
+ }
+ String dynamicProperty = properties.getProperty("dynamic.property");
+ if (dynamicProperty != null) {
+ String namespace = this.getWindowNameSpace();
+ String name = this.getWindowName();
+ String startTime = this.getStartTime();
+ String endTime = this.getEndTime();
+ String startTimeStr = startTime.replace("-", "").replace(" ", "").replace(":", "");
+ String endTimeStr = endTime.replace("-", "").replace(" ", "").replace(":", "");
+ String dynamicPropertyValue = MapKeyUtil.createKeyBySign("_", namespace, name, startTimeStr + "",
+ endTimeStr + "");
+ dynamicPropertyValue = dynamicPropertyValue.replaceAll("\\.", "_");
+ String[] mutilPropertys = dynamicProperty.split(",");
+ String groupName = MapKeyUtil.createKeyBySign("_", namespace, name).replaceAll("\\.", "_");
+ for (String properyKey : mutilPropertys) {
+ if (properyKey.equals("group")) {
+ properties.put(properyKey, groupName);
+ } else {
+ properties.put(properyKey, dynamicPropertyValue);
+ }
+
+ }
+ }
+ return properties;
+ }
+
+ public String getWindowNameSpace() {
+ return "windowNameSpace";
+ }
+
+ public String getWindowName() {
+ return "windowName";
+ }
+
+ public String getStartTime() {
+ return "startTime";
+ }
+
+ public String getEndTime() {
+ return "endTime";
+ }
+
+ @Override
+ protected ISource createSource() {
+ RocketMQChannelBuilder builder = new RocketMQChannelBuilder();
+ ISource rocketMQSource = builder.createSource(getWindowNameSpace(), getWindowName(), createChannelProperties(), null);
+ // RocketMQSource rocketMQSource = new RocketMQSource();
+ // rocketMQSource.setTopic("TOPIC_DIPPER_WINDOW_STATISTICS");
+ // rocketMQSource.setTags("test");
+ // rocketMQSource.setAccessKey();
+ return rocketMQSource;
+ }
+
+ @Override
+ protected ISink createSink() {
+ RocketMQChannelBuilder builder = new RocketMQChannelBuilder();
+ ISink sink = builder.createSink(getWindowNameSpace(), getWindowName(), createChannelProperties(), null);
+ return sink;
+ }
+
+}
diff --git a/rocketmq-streams-channel-rocketmq/target/classes/META-INF/services/org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder b/rocketmq-streams-channel-rocketmq/target/classes/META-INF/services/org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder
new file mode 100644
index 0000000..fb9e053
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/target/classes/META-INF/services/org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder
@@ -0,0 +1 @@
+org.apache.rocketmq.streams.RocketMQChannelBuilder
diff --git a/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/RocketMQChannelBuilder.class b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/RocketMQChannelBuilder.class
new file mode 100644
index 0000000..bcd8f08
Binary files /dev/null and b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/RocketMQChannelBuilder.class differ
diff --git a/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/RocketMQOffset.class b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/RocketMQOffset.class
new file mode 100644
index 0000000..b1f45cf
Binary files /dev/null and b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/RocketMQOffset.class differ
diff --git a/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/queue/RocketMQMessageQueue.class b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/queue/RocketMQMessageQueue.class
new file mode 100644
index 0000000..a9c8a39
Binary files /dev/null and b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/queue/RocketMQMessageQueue.class differ
diff --git a/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/sink/RocketMQSink$1.class b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/sink/RocketMQSink$1.class
new file mode 100644
index 0000000..b2f09df
Binary files /dev/null and b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/sink/RocketMQSink$1.class differ
diff --git a/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/sink/RocketMQSink.class b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/sink/RocketMQSink.class
new file mode 100644
index 0000000..12fbfb1
Binary files /dev/null and b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/sink/RocketMQSink.class differ
diff --git a/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/source/RocketMQSource$1.class b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/source/RocketMQSource$1.class
new file mode 100644
index 0000000..e9fe35c
Binary files /dev/null and b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/source/RocketMQSource$1.class differ
diff --git a/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/source/RocketMQSource.class b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/source/RocketMQSource.class
new file mode 100644
index 0000000..e186384
Binary files /dev/null and b/rocketmq-streams-channel-rocketmq/target/classes/org/apache/rocketmq/streams/source/RocketMQSource.class differ
diff --git a/rocketmq-streams-channel-rocketmq/target/maven-archiver/pom.properties b/rocketmq-streams-channel-rocketmq/target/maven-archiver/pom.properties
new file mode 100644
index 0000000..a605b53
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/target/maven-archiver/pom.properties
@@ -0,0 +1,5 @@
+#Generated by Maven
+#Fri Jul 30 11:10:37 CST 2021
+version=2.0.0-SNAPSHOT
+groupId=org.apache.rocketmq
+artifactId=rocketmq-streams-channel-rocketmq
diff --git a/rocketmq-streams-channel-rocketmq/target/rocketmq-streams-channel-rocketmq-2.0.0-SNAPSHOT-sources.jar b/rocketmq-streams-channel-rocketmq/target/rocketmq-streams-channel-rocketmq-2.0.0-SNAPSHOT-sources.jar
new file mode 100644
index 0000000..6a2ed7f
Binary files /dev/null and b/rocketmq-streams-channel-rocketmq/target/rocketmq-streams-channel-rocketmq-2.0.0-SNAPSHOT-sources.jar differ
diff --git a/rocketmq-streams-channel-rocketmq/target/rocketmq-streams-channel-rocketmq-2.0.0-SNAPSHOT.jar b/rocketmq-streams-channel-rocketmq/target/rocketmq-streams-channel-rocketmq-2.0.0-SNAPSHOT.jar
new file mode 100644
index 0000000..333a58d
Binary files /dev/null and b/rocketmq-streams-channel-rocketmq/target/rocketmq-streams-channel-rocketmq-2.0.0-SNAPSHOT.jar differ
diff --git a/rocketmq-streams-serviceloader/pom.xml b/rocketmq-streams-serviceloader/pom.xml
new file mode 100755
index 0000000..f48f581
--- /dev/null
+++ b/rocketmq-streams-serviceloader/pom.xml
@@ -0,0 +1,24 @@
+<?xml version="1.0" encoding="utf-8"?>
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.rocketmq</groupId>
+ <artifactId>rocketmq-streams</artifactId>
+ <version>2.0.0-SNAPSHOT</version>
+ </parent>
+ <artifactId>rocketmq-streams-serviceloader</artifactId>
+ <packaging>jar</packaging>
+ <name>ROCKETMQ STREAMS :: serviceloader</name>
+ <dependencies>
+
+ <!-- 测试依赖 -->
+ <dependency>
+ <groupId>org.apache.rocketmq</groupId>
+ <artifactId>rocketmq-streams-commons</artifactId>
+ </dependency>
+
+ </dependencies>
+</project>
diff --git a/rocketmq-streams-serviceloader/rocketmq-streams-serviceloader.iml b/rocketmq-streams-serviceloader/rocketmq-streams-serviceloader.iml
new file mode 100644
index 0000000..5a29414
--- /dev/null
+++ b/rocketmq-streams-serviceloader/rocketmq-streams-serviceloader.iml
@@ -0,0 +1,61 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<module org.jetbrains.idea.maven.project.MavenProjectsManager.isMavenModule="true" type="JAVA_MODULE" version="4">
+ <component name="NewModuleRootManager" LANGUAGE_LEVEL="JDK_1_8">
+ <output url="file://$MODULE_DIR$/target/classes" />
+ <output-test url="file://$MODULE_DIR$/target/test-classes" />
+ <content url="file://$MODULE_DIR$">
+ <sourceFolder url="file://$MODULE_DIR$/src/main/java" isTestSource="false" />
+ <sourceFolder url="file://$MODULE_DIR$/src/main/resources" type="java-resource" />
+ <sourceFolder url="file://$MODULE_DIR$/src/test/java" isTestSource="true" />
+ <sourceFolder url="file://$MODULE_DIR$/src/test/resources" type="java-test-resource" />
+ <excludeFolder url="file://$MODULE_DIR$/target" />
+ </content>
+ <orderEntry type="inheritedJdk" />
+ <orderEntry type="sourceFolder" forTests="false" />
+ <orderEntry type="module" module-name="rocketmq-streams-commons" />
+ <orderEntry type="library" name="Maven: org.apache.commons:commons-lang3:3.11" level="project" />
+ <orderEntry type="library" name="Maven: junit:junit:4.12" level="project" />
+ <orderEntry type="library" name="Maven: org.hamcrest:hamcrest-core:1.3" level="project" />
+ <orderEntry type="library" name="Maven: com.alibaba:fastjson:1.2.27" level="project" />
+ <orderEntry type="library" name="Maven: commons-logging:commons-logging:1.1" level="project" />
+ <orderEntry type="library" name="Maven: logkit:logkit:1.0.1" level="project" />
+ <orderEntry type="library" name="Maven: avalon-framework:avalon-framework:4.1.3" level="project" />
+ <orderEntry type="library" name="Maven: javax.servlet:servlet-api:2.3" level="project" />
+ <orderEntry type="library" name="Maven: commons-io:commons-io:2.5" level="project" />
+ <orderEntry type="library" name="Maven: log4j:log4j:1.2.17" level="project" />
+ <orderEntry type="library" name="Maven: com.google.code.gson:gson:2.8.5" level="project" />
+ <orderEntry type="library" name="Maven: com.google.auto.service:auto-service:1.0-rc5" level="project" />
+ <orderEntry type="library" name="Maven: com.google.auto.service:auto-service-annotations:1.0-rc5" level="project" />
+ <orderEntry type="library" name="Maven: com.google.auto:auto-common:0.10" level="project" />
+ <orderEntry type="library" name="Maven: com.google.guava:guava:25.1-jre" level="project" />
+ <orderEntry type="library" name="Maven: com.google.code.findbugs:jsr305:3.0.2" level="project" />
+ <orderEntry type="library" name="Maven: org.checkerframework:checker-qual:2.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.google.errorprone:error_prone_annotations:2.1.3" level="project" />
+ <orderEntry type="library" name="Maven: com.google.j2objc:j2objc-annotations:1.1" level="project" />
+ <orderEntry type="library" name="Maven: org.codehaus.mojo:animal-sniffer-annotations:1.14" level="project" />
+ <orderEntry type="library" name="Maven: com.lmax:disruptor:3.2.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:hyperscan:5.4.0-2.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:linux-x86_64:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:windows-x86_64:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:macosx-x86_64:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp-platform:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-arm:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-arm64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-x86:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:ios-arm64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:ios-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-armhf:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-arm64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-ppc64le:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-x86:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:macosx-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:windows-x86:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:windows-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: net.java.dev.jna:platform:3.5.2" level="project" />
+ <orderEntry type="library" name="Maven: net.java.dev.jna:jna:3.5.2" level="project" />
+ </component>
+</module>
\ No newline at end of file
diff --git a/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/IServiceLoaderService.java b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/IServiceLoaderService.java
new file mode 100644
index 0000000..1e63c38
--- /dev/null
+++ b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/IServiceLoaderService.java
@@ -0,0 +1,31 @@
+package org.apache.rocketmq.streams.serviceloader;
+
+import java.util.List;
+
+public interface IServiceLoaderService<T> {
+
+ /**
+ * 获取某个指定名称的服务对象
+ *
+ * @param name
+ * @param
+ * @return
+ */
+ T loadService(String name);
+
+ /**
+ * 获取多个实现类
+ *
+ * @param
+ * @return
+ */
+ List<T> loadService();
+
+ /**
+ * 如果forceRefresh==false, refresh多次调用只会执行一次; 如果forceRefresh==true,每次调用都会做一次重新扫描
+ *
+ * @param forceRefresh
+ */
+ void refresh(boolean forceRefresh);
+
+}
diff --git a/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponent.java b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponent.java
new file mode 100644
index 0000000..847c033
--- /dev/null
+++ b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponent.java
@@ -0,0 +1,132 @@
+package org.apache.rocketmq.streams.serviceloader;
+
+import org.apache.rocketmq.streams.common.component.ComponentCreator;
+import org.apache.rocketmq.streams.common.model.ServiceName;
+import org.apache.rocketmq.streams.serviceloader.namefinder.IServiceNameGetter;
+import org.apache.rocketmq.streams.common.component.AbstractComponent;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+
+import java.util.*;
+
+public class ServiceLoaderComponent<T> extends AbstractComponent<IServiceLoaderService<T>>
+ implements IServiceLoaderService<T> {
+ private Properties properties;
+ private Class<T> interfaceClass;
+ private Map<String, T> name2Service = new HashMap<>();
+ private List<T> serviceList = new ArrayList<T>();
+ private boolean hasRefresh = false;
+ private boolean needServieName = true;
+
+ @Override
+ public boolean stop() {
+ return true;
+ }
+
+ @Override
+ public IServiceLoaderService<T> getService() {
+ return this;
+ }
+
+ public static ServiceLoaderComponent getInstance(Class interfaceClass) {
+ ServiceLoaderComponent serviceLoaderComponent =
+ ComponentCreator.getComponent(interfaceClass.getName(), ServiceLoaderComponent.class);
+ return serviceLoaderComponent;
+ }
+
+ @Override
+ protected boolean startComponent(String interfaceClassName) {
+ try {
+ Class clazz = Class.forName(interfaceClassName);
+ this.interfaceClass = clazz;
+ refresh(false);
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException("class not found " + interfaceClassName, e);
+ }
+ return true;
+ }
+
+ @Override
+ protected boolean initProperties(Properties properties) {
+ this.properties = properties;
+ return true;
+ }
+
+ @Override
+ public T loadService(String serviceName) {
+ if (!this.hasRefresh) {
+ refresh(false);
+ }
+ return (T)this.name2Service.get(serviceName);
+ }
+
+ @Override
+ public List<T> loadService() {
+ if (!this.hasRefresh) {
+ refresh(false);
+ }
+ return serviceList;
+ }
+
+ @Override
+ public void refresh(boolean forceRefresh) {
+ if (!forceRefresh && hasRefresh) {
+ return;
+ }
+ synchronized (this) {
+ if (!forceRefresh && hasRefresh) {
+ return;
+ }
+ Map<String, T> name2Service = new HashMap<>();
+ Iterable<T> iterable = ServiceLoader.load(interfaceClass);
+ List<T> allService = new ArrayList<>();
+ for (T t : iterable) {
+ if (needServieName) {
+ List<String> serviceNames = loadServiceName(t);
+ if (serviceNames == null) {
+ name2Service.put(t.getClass().getSimpleName(), t);
+ } else {
+ for (String serviceName : serviceNames) {
+ name2Service.put(serviceName, t);
+ }
+ }
+
+ }
+ allService.add(t);
+ }
+ this.name2Service = name2Service;
+ this.hasRefresh = true;
+ this.serviceList = allService;
+ }
+ }
+
+ static ServiceLoaderComponent nameLoaderComponent = new ServiceLoaderComponent();
+
+ static {
+ nameLoaderComponent.init();
+ nameLoaderComponent.needServieName = true;
+ nameLoaderComponent.startComponent(IServiceNameGetter.class.getName());
+ }
+
+ protected List<String> loadServiceName(T t) {
+ List<String> serviceNames = new ArrayList();
+ Class tClass = t.getClass();
+ String serviceName = properties.getProperty(tClass.getName());
+ if (properties != null && StringUtil.isNotEmpty(serviceName)) {
+
+ serviceNames.add(serviceName);
+ return serviceNames;
+ }
+ ServiceName annotation = (ServiceName)tClass.getAnnotation(ServiceName.class);
+ if (annotation == null) {
+ return null;
+ }
+ if (StringUtil.isNotEmpty(annotation.value())) {
+ serviceNames.add(annotation.value());
+ }
+ if (StringUtil.isNotEmpty(annotation.aliasName())) {
+ serviceNames.add(annotation.aliasName());
+ }
+
+ return serviceNames;
+ }
+}
diff --git a/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/namefinder/IServiceNameGetter.java b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/namefinder/IServiceNameGetter.java
new file mode 100644
index 0000000..2753b5b
--- /dev/null
+++ b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/namefinder/IServiceNameGetter.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.serviceloader.namefinder;
+
+/**
+ * 获取service
+ */
+public interface IServiceNameGetter {
+ /**
+ * 根据class获取class的name
+ *
+ * @param clazz
+ * @return
+ */
+ String getServiceName(Class clazz);
+}
diff --git a/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/namefinder/impl/AnnotationServiceNameGetter.java b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/namefinder/impl/AnnotationServiceNameGetter.java
new file mode 100644
index 0000000..803e800
--- /dev/null
+++ b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/namefinder/impl/AnnotationServiceNameGetter.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.serviceloader.namefinder.impl;
+
+import org.apache.rocketmq.streams.common.model.ServiceName;
+import org.apache.rocketmq.streams.serviceloader.namefinder.IServiceNameGetter;
+import com.google.auto.service.AutoService;
+
+/**
+ * 根据class的标注获取servicename
+ */
+@AutoService(IServiceNameGetter.class)
+@ServiceName(AnnotationServiceNameGetter.SERVICE_NAME)
+public class AnnotationServiceNameGetter implements IServiceNameGetter {
+ public static final String SERVICE_NAME = "SERVICE_ANNOTATION";
+
+ @Override
+ public String getServiceName(Class clazz) {
+ ServiceName annotation = (ServiceName)clazz.getAnnotation(ServiceName.class);
+ if (annotation == null) {
+ return null;
+ }
+ return annotation.value();
+ }
+}
diff --git a/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/utils/FileUtil.java b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/utils/FileUtil.java
new file mode 100644
index 0000000..cc2efaf
--- /dev/null
+++ b/rocketmq-streams-serviceloader/src/main/java/org/apache/rocketmq/streams/serviceloader/utils/FileUtil.java
@@ -0,0 +1,73 @@
+package org.apache.rocketmq.streams.serviceloader.utils;
+
+import com.google.common.io.Closer;
+
+import java.io.*;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+public class FileUtil extends org.apache.rocketmq.streams.common.utils.FileUtil {
+ public static final String SERVICES_PATH = "META-INF/services";
+
+ /**
+ * Returns an absolute path to a service file given the class name of the service.
+ *
+ * @param serviceName not {@code null}
+ * @return SERVICES_PATH + serviceName
+ */
+ static String getPath(String serviceName) {
+ return SERVICES_PATH + "/" + serviceName;
+ }
+
+ /**
+ * Reads the set of service classes from a service file.
+ *
+ * @param input not {@code null}. Closed after use.
+ * @return a not {@code null Set} of service class names.
+ * @throws IOException
+ */
+ public static Set<String> readServiceFile(InputStream input) throws IOException {
+ HashSet<String> serviceClasses = new HashSet<String>();
+ Closer closer = Closer.create();
+ try {
+ // TODO(gak): use CharStreams
+ BufferedReader r = closer.register(new BufferedReader(new InputStreamReader(input, UTF_8)));
+ String line;
+ while ((line = r.readLine()) != null) {
+ int commentStart = line.indexOf('#');
+ if (commentStart >= 0) {
+ line = line.substring(0, commentStart);
+ }
+ line = line.trim();
+ if (!line.isEmpty()) {
+ serviceClasses.add(line);
+ }
+ }
+ return serviceClasses;
+ } catch (Throwable t) {
+ throw closer.rethrow(t);
+ } finally {
+ closer.close();
+ }
+ }
+
+ /**
+ * Writes the set of service class names to a service file.
+ *
+ * @param output not {@code null}. Not closed after use.
+ * @param services a not {@code null Collection} of service class names.
+ * @throws IOException
+ */
+ public static void writeServiceFile(Collection<String> services, OutputStream output) throws IOException {
+ BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(output, UTF_8));
+ for (String service : services) {
+ writer.write(service);
+ writer.newLine();
+ }
+ writer.flush();
+ }
+}
+
diff --git a/rocketmq-streams-serviceloader/src/main/resources/component/ServiceLoaderComponent.properties b/rocketmq-streams-serviceloader/src/main/resources/component/ServiceLoaderComponent.properties
new file mode 100644
index 0000000..0f1b2bb
--- /dev/null
+++ b/rocketmq-streams-serviceloader/src/main/resources/component/ServiceLoaderComponent.properties
@@ -0,0 +1 @@
+#serviceName=class.getName()
\ No newline at end of file
diff --git a/rocketmq-streams-serviceloader/src/test/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponentTest.java b/rocketmq-streams-serviceloader/src/test/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponentTest.java
new file mode 100644
index 0000000..b68ea36
--- /dev/null
+++ b/rocketmq-streams-serviceloader/src/test/java/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponentTest.java
@@ -0,0 +1,17 @@
+package org.apache.rocketmq.streams.serviceloader;
+
+import org.apache.rocketmq.streams.serviceloader.namefinder.IServiceNameGetter;
+import org.apache.rocketmq.streams.serviceloader.namefinder.impl.AnnotationServiceNameGetter;
+import org.junit.Test;
+
+import static junit.framework.TestCase.assertTrue;
+
+public class ServiceLoaderComponentTest {
+
+ @Test
+ public void testLoadService() {
+ ServiceLoaderComponent serviceLoaderComponent = ServiceLoaderComponent.getInstance(IServiceNameGetter.class);
+ AnnotationServiceNameGetter getter = (AnnotationServiceNameGetter)serviceLoaderComponent.getService().loadService(AnnotationServiceNameGetter.SERVICE_NAME);
+ assertTrue(getter != null && AnnotationServiceNameGetter.class.isInstance(getter));
+ }
+}
diff --git a/rocketmq-streams-serviceloader/src/test/resources/log4j.xml b/rocketmq-streams-serviceloader/src/test/resources/log4j.xml
new file mode 100755
index 0000000..7812fe7
--- /dev/null
+++ b/rocketmq-streams-serviceloader/src/test/resources/log4j.xml
@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!DOCTYPE log4j:configuration SYSTEM "http://toolkit.alibaba-inc.com/dtd/log4j/log4j.dtd">
+<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
+
+ <appender name="Console" class="org.apache.log4j.ConsoleAppender">
+ <layout class="org.apache.log4j.PatternLayout">
+ <param name="ConversionPattern" value="%d{ISO8601} %l [%t] %-5p - %m%n%n"/>
+ </layout>
+ <filter class="org.apache.log4j.varia.LevelRangeFilter">
+ <param name="LevelMin" value="INFO"/>
+ <param name="LevelMax" value="ERROR"/>
+ </filter>
+ </appender>
+
+ <root>
+ <priority value="INFO"/>
+ <appender-ref ref="Console"/>
+ </root>
+
+</log4j:configuration>
\ No newline at end of file
diff --git a/rocketmq-streams-serviceloader/target/classes/META-INF/services/org.apache.rocketmq.streams.serviceloader.namefinder.IServiceNameGetter b/rocketmq-streams-serviceloader/target/classes/META-INF/services/org.apache.rocketmq.streams.serviceloader.namefinder.IServiceNameGetter
new file mode 100644
index 0000000..30fdc46
--- /dev/null
+++ b/rocketmq-streams-serviceloader/target/classes/META-INF/services/org.apache.rocketmq.streams.serviceloader.namefinder.IServiceNameGetter
@@ -0,0 +1 @@
+org.apache.rocketmq.streams.serviceloader.namefinder.impl.AnnotationServiceNameGetter
diff --git a/rocketmq-streams-serviceloader/target/classes/component/ServiceLoaderComponent.properties b/rocketmq-streams-serviceloader/target/classes/component/ServiceLoaderComponent.properties
new file mode 100644
index 0000000..0f1b2bb
--- /dev/null
+++ b/rocketmq-streams-serviceloader/target/classes/component/ServiceLoaderComponent.properties
@@ -0,0 +1 @@
+#serviceName=class.getName()
\ No newline at end of file
diff --git a/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/IServiceLoaderService.class b/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/IServiceLoaderService.class
new file mode 100644
index 0000000..bb8c29e
Binary files /dev/null and b/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/IServiceLoaderService.class differ
diff --git a/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponent.class b/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponent.class
new file mode 100644
index 0000000..3831e75
Binary files /dev/null and b/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponent.class differ
diff --git a/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/namefinder/IServiceNameGetter.class b/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/namefinder/IServiceNameGetter.class
new file mode 100644
index 0000000..84de58a
Binary files /dev/null and b/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/namefinder/IServiceNameGetter.class differ
diff --git a/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/namefinder/impl/AnnotationServiceNameGetter.class b/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/namefinder/impl/AnnotationServiceNameGetter.class
new file mode 100644
index 0000000..193c479
Binary files /dev/null and b/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/namefinder/impl/AnnotationServiceNameGetter.class differ
diff --git a/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/utils/FileUtil.class b/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/utils/FileUtil.class
new file mode 100644
index 0000000..e3b051c
Binary files /dev/null and b/rocketmq-streams-serviceloader/target/classes/org/apache/rocketmq/streams/serviceloader/utils/FileUtil.class differ
diff --git a/rocketmq-streams-serviceloader/target/maven-archiver/pom.properties b/rocketmq-streams-serviceloader/target/maven-archiver/pom.properties
new file mode 100644
index 0000000..037c58d
--- /dev/null
+++ b/rocketmq-streams-serviceloader/target/maven-archiver/pom.properties
@@ -0,0 +1,5 @@
+#Generated by Maven
+#Fri Jul 30 11:10:32 CST 2021
+version=2.0.0-SNAPSHOT
+groupId=org.apache.rocketmq
+artifactId=rocketmq-streams-serviceloader
diff --git a/rocketmq-streams-serviceloader/target/rocketmq-streams-serviceloader-2.0.0-SNAPSHOT-sources.jar b/rocketmq-streams-serviceloader/target/rocketmq-streams-serviceloader-2.0.0-SNAPSHOT-sources.jar
new file mode 100644
index 0000000..1695e65
Binary files /dev/null and b/rocketmq-streams-serviceloader/target/rocketmq-streams-serviceloader-2.0.0-SNAPSHOT-sources.jar differ
diff --git a/rocketmq-streams-serviceloader/target/rocketmq-streams-serviceloader-2.0.0-SNAPSHOT.jar b/rocketmq-streams-serviceloader/target/rocketmq-streams-serviceloader-2.0.0-SNAPSHOT.jar
new file mode 100644
index 0000000..9dc634b
Binary files /dev/null and b/rocketmq-streams-serviceloader/target/rocketmq-streams-serviceloader-2.0.0-SNAPSHOT.jar differ
diff --git a/rocketmq-streams-serviceloader/target/test-classes/log4j.xml b/rocketmq-streams-serviceloader/target/test-classes/log4j.xml
new file mode 100755
index 0000000..7812fe7
--- /dev/null
+++ b/rocketmq-streams-serviceloader/target/test-classes/log4j.xml
@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!DOCTYPE log4j:configuration SYSTEM "http://toolkit.alibaba-inc.com/dtd/log4j/log4j.dtd">
+<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
+
+ <appender name="Console" class="org.apache.log4j.ConsoleAppender">
+ <layout class="org.apache.log4j.PatternLayout">
+ <param name="ConversionPattern" value="%d{ISO8601} %l [%t] %-5p - %m%n%n"/>
+ </layout>
+ <filter class="org.apache.log4j.varia.LevelRangeFilter">
+ <param name="LevelMin" value="INFO"/>
+ <param name="LevelMax" value="ERROR"/>
+ </filter>
+ </appender>
+
+ <root>
+ <priority value="INFO"/>
+ <appender-ref ref="Console"/>
+ </root>
+
+</log4j:configuration>
\ No newline at end of file
diff --git a/rocketmq-streams-serviceloader/target/test-classes/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponentTest.class b/rocketmq-streams-serviceloader/target/test-classes/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponentTest.class
new file mode 100644
index 0000000..100837f
Binary files /dev/null and b/rocketmq-streams-serviceloader/target/test-classes/org/apache/rocketmq/streams/serviceloader/ServiceLoaderComponentTest.class differ
diff --git a/rocketmq-streams-window/pom.xml b/rocketmq-streams-window/pom.xml
new file mode 100755
index 0000000..8c5a709
--- /dev/null
+++ b/rocketmq-streams-window/pom.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="utf-8"?>
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.rocketmq</groupId>
+ <artifactId>rocketmq-streams</artifactId>
+ <version>2.0.0-SNAPSHOT</version>
+ </parent>
+ <artifactId>rocketmq-streams-window</artifactId>
+ <packaging>jar</packaging>
+ <name>ROCKETMQ STREAMS :: window</name>
+ <dependencies>
+
+ <dependency>
+ <groupId>org.apache.rocketmq</groupId>
+ <artifactId>rocketmq-streams-db-operator</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.rocketmq</groupId>
+ <artifactId>rocketmq-streams-lease</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.rocketmq</groupId>
+ <artifactId>rocketmq-streams-dim</artifactId>
+ </dependency>
+
+
+ <dependency>
+ <groupId>org.rocksdb</groupId>
+ <artifactId>rocksdbjni</artifactId>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/rocketmq-streams-window/rocketmq-streams-window.iml b/rocketmq-streams-window/rocketmq-streams-window.iml
new file mode 100644
index 0000000..793fdfa
--- /dev/null
+++ b/rocketmq-streams-window/rocketmq-streams-window.iml
@@ -0,0 +1,82 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<module org.jetbrains.idea.maven.project.MavenProjectsManager.isMavenModule="true" type="JAVA_MODULE" version="4">
+ <component name="NewModuleRootManager" LANGUAGE_LEVEL="JDK_1_8">
+ <output url="file://$MODULE_DIR$/target/classes" />
+ <output-test url="file://$MODULE_DIR$/target/test-classes" />
+ <content url="file://$MODULE_DIR$">
+ <sourceFolder url="file://$MODULE_DIR$/src/main/java" isTestSource="false" />
+ <sourceFolder url="file://$MODULE_DIR$/src/main/resources" type="java-resource" />
+ <sourceFolder url="file://$MODULE_DIR$/src/test/java" isTestSource="true" />
+ <sourceFolder url="file://$MODULE_DIR$/src/test/resources" type="java-test-resource" />
+ <excludeFolder url="file://$MODULE_DIR$/target" />
+ </content>
+ <orderEntry type="inheritedJdk" />
+ <orderEntry type="sourceFolder" forTests="false" />
+ <orderEntry type="module" module-name="rocketmq-streams-db-operator" />
+ <orderEntry type="module" module-name="rocketmq-streams-configurable" />
+ <orderEntry type="module" module-name="rocketmq-streams-serviceloader" />
+ <orderEntry type="library" name="Maven: org.springframework:spring-jdbc:3.2.13.RELEASE" level="project" />
+ <orderEntry type="library" name="Maven: org.springframework:spring-beans:3.2.13.RELEASE" level="project" />
+ <orderEntry type="library" name="Maven: org.springframework:spring-core:3.2.13.RELEASE" level="project" />
+ <orderEntry type="library" name="Maven: commons-logging:commons-logging:1.1" level="project" />
+ <orderEntry type="library" name="Maven: logkit:logkit:1.0.1" level="project" />
+ <orderEntry type="library" name="Maven: avalon-framework:avalon-framework:4.1.3" level="project" />
+ <orderEntry type="library" name="Maven: javax.servlet:servlet-api:2.3" level="project" />
+ <orderEntry type="library" name="Maven: org.springframework:spring-tx:3.2.13.RELEASE" level="project" />
+ <orderEntry type="library" name="Maven: mysql:mysql-connector-java:5.1.40" level="project" />
+ <orderEntry type="module" module-name="rocketmq-streams-lease" />
+ <orderEntry type="module" module-name="rocketmq-streams-channel-db" />
+ <orderEntry type="module" module-name="rocketmq-streams-dim" />
+ <orderEntry type="module" module-name="rocketmq-streams-filter" />
+ <orderEntry type="module" module-name="rocketmq-streams-script" />
+ <orderEntry type="library" name="Maven: io.krakens:java-grok:0.1.9" level="project" />
+ <orderEntry type="library" name="Maven: org.codehaus.groovy:groovy-all:2.1.8" level="project" />
+ <orderEntry type="library" name="Maven: org.python:jython-standalone:2.7.0" level="project" />
+ <orderEntry type="module" module-name="rocketmq-streams-channel-http" />
+ <orderEntry type="module" module-name="rocketmq-streams-commons" />
+ <orderEntry type="library" name="Maven: org.apache.commons:commons-lang3:3.11" level="project" />
+ <orderEntry type="library" name="Maven: junit:junit:4.12" level="project" />
+ <orderEntry type="library" name="Maven: org.hamcrest:hamcrest-core:1.3" level="project" />
+ <orderEntry type="library" name="Maven: com.alibaba:fastjson:1.2.27" level="project" />
+ <orderEntry type="library" name="Maven: log4j:log4j:1.2.17" level="project" />
+ <orderEntry type="library" name="Maven: com.google.code.gson:gson:2.8.5" level="project" />
+ <orderEntry type="library" name="Maven: com.google.auto.service:auto-service:1.0-rc5" level="project" />
+ <orderEntry type="library" name="Maven: com.google.auto.service:auto-service-annotations:1.0-rc5" level="project" />
+ <orderEntry type="library" name="Maven: com.google.auto:auto-common:0.10" level="project" />
+ <orderEntry type="library" name="Maven: com.google.guava:guava:25.1-jre" level="project" />
+ <orderEntry type="library" name="Maven: com.google.code.findbugs:jsr305:3.0.2" level="project" />
+ <orderEntry type="library" name="Maven: org.checkerframework:checker-qual:2.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.google.errorprone:error_prone_annotations:2.1.3" level="project" />
+ <orderEntry type="library" name="Maven: com.google.j2objc:j2objc-annotations:1.1" level="project" />
+ <orderEntry type="library" name="Maven: org.codehaus.mojo:animal-sniffer-annotations:1.14" level="project" />
+ <orderEntry type="library" name="Maven: com.lmax:disruptor:3.2.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:hyperscan:5.4.0-2.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:linux-x86_64:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:windows-x86_64:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: com.gliwka.hyperscan:native:macosx-x86_64:5.4.0-1.0.0" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp-platform:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-arm:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-arm64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-x86:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:android-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:ios-arm64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:ios-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-armhf:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-arm64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-ppc64le:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-x86:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:linux-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:macosx-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:windows-x86:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: org.bytedeco:javacpp:windows-x86_64:1.5.4" level="project" />
+ <orderEntry type="library" name="Maven: net.java.dev.jna:platform:3.5.2" level="project" />
+ <orderEntry type="library" name="Maven: net.java.dev.jna:jna:3.5.2" level="project" />
+ <orderEntry type="library" name="Maven: commons-io:commons-io:2.5" level="project" />
+ <orderEntry type="library" name="Maven: org.apache.httpcomponents:httpclient:4.5.2" level="project" />
+ <orderEntry type="library" name="Maven: org.apache.httpcomponents:httpcore:4.4.4" level="project" />
+ <orderEntry type="library" name="Maven: commons-codec:commons-codec:1.9" level="project" />
+ <orderEntry type="library" name="Maven: org.rocksdb:rocksdbjni:6.6.4" level="project" />
+ </component>
+</module>
\ No newline at end of file
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
new file mode 100644
index 0000000..62025b8
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java
@@ -0,0 +1,95 @@
+/*
+ * 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.builder;
+
+import org.apache.rocketmq.streams.window.operator.AbstractWindow;
+import org.apache.rocketmq.streams.window.operator.impl.WindowOperator;
+import org.apache.rocketmq.streams.common.component.ComponentCreator;
+import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+import org.apache.rocketmq.streams.window.operator.join.JoinWindow;
+import org.apache.rocketmq.streams.window.operator.impl.OverWindow;
+
+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 void openTestModel() {
+ TEST_MODE = true;
+ }
+
+ public static void closeTestModel() {
+ TEST_MODE = false;
+ }
+
+ /**
+ * 创建join的窗口对象,join的窗口大小可以配置文件配置,如果未配置用写死的默认值
+ *
+ * @return
+ */
+ public static JoinWindow createDefaultJoinWindow() {
+ JoinWindow joinWindow = new JoinWindow();
+ if (TEST_MODE) {
+ joinWindow = new JoinWindow();
+ }
+ joinWindow.setSizeInterval(getIntValue(ConfigureFileKey.DIPPER_WINDOW_JOIN_DEFAULT_ITERVA_SIZE, 5));//默认5分钟一个窗口
+ joinWindow.setRetainWindowCount(getIntValue(ConfigureFileKey.DIPPER_WINDOW_JOIN_RETAIN_WINDOW_COUNT, 6));//join的时间窗口是20分钟
+ joinWindow.setWindowType(AbstractWindow.TUMBLE_WINDOW);
+ // joinWindow.setFireDelaySecond(getIntValue(ConfigureFileKey.DIPPER_WINDOW_DEFAULT_FIRE_DELAY_SECOND,5));//延迟5分钟触发
+ joinWindow.setTimeFieldName("");
+ joinWindow.setSlideInterval(getIntValue(ConfigureFileKey.DIPPER_WINDOW_JOIN_DEFAULT_ITERVA_SIZE, 5));
+ joinWindow.setWaterMarkMinute(0);
+ joinWindow.setWindowType(AbstractWindow.TUMBLE_WINDOW);
+ return joinWindow;
+ }
+
+ public static OverWindow createOvertWindow(String groupBy, String rowNumName) {
+ OverWindow overWindow = new OverWindow();
+ overWindow.setGroupByFieldName(groupBy);
+ overWindow.setRowNumerName(rowNumName);
+ overWindow.setTimeFieldName("");
+ overWindow.setSizeInterval(getIntValue(ConfigureFileKey.DIPPER_WINDOW_OVER_DEFAULT_ITERVA_SIZE, 60));
+ overWindow.setSlideInterval(overWindow.getSizeInterval());
+ return overWindow;
+ }
+
+ /**
+ * 获取配置配置文件的值,如果配置文件为配置,则用默认值
+ *
+ * @param propertyKey
+ * @param defalutValue
+ * @return
+ */
+ public static int getIntValue(String propertyKey, int defalutValue) {
+ String value = ComponentCreator.getProperties().getProperty(propertyKey);
+ if (StringUtil.isNotEmpty(value)) {
+ return Integer.valueOf(value);
+ }
+ return defalutValue;
+ }
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/FunctionExecutor.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/FunctionExecutor.java
new file mode 100644
index 0000000..a6da407
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/FunctionExecutor.java
@@ -0,0 +1,51 @@
+/*
+ * 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.model;
+
+import java.util.List;
+
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.interfaces.IStreamOperator;
+
+public class FunctionExecutor {
+
+ private static final String KEY = "key";
+
+ /**
+ * the computed column defined by user or system
+ */
+ private String column;
+
+ /**
+ * the executor of operator(column=function(xxx))
+ */
+ private IStreamOperator<IMessage, List<IMessage>> executor;
+
+ public FunctionExecutor(String column, IStreamOperator<IMessage, List<IMessage>> executor) {
+ this.column = column;
+ this.executor = executor;
+ }
+
+ public String getColumn() {
+ return column;
+ }
+
+ public IStreamOperator<IMessage, List<IMessage>> getExecutor() {
+ return executor;
+ }
+
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowCache.java
new file mode 100644
index 0000000..3ac082d
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowCache.java
@@ -0,0 +1,149 @@
+/*
+ * 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.model;
+
+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 com.alibaba.fastjson.JSONArray;
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.common.channel.sink.AbstractSink;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.channel.IChannel;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+import org.apache.rocketmq.streams.window.shuffle.ShuffleChannel;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * 缓存数据,flush时,刷新完成数据落盘
+ */
+public abstract class WindowCache extends
+ AbstractSink {
+
+ private static final Log LOG = LogFactory.getLog(WindowCache.class);
+
+ public static final String SPLIT_SIGN = "##";
+
+ public static final String ORIGIN_OFFSET = "origin_offset";
+
+ public static final String ORIGIN_QUEUE_ID = "origin_queue_id";
+
+ public static final String ORIGIN_MESSAGE_HEADER = "origin_message_header";
+
+ public static final String SHUFFLE_KEY = "SHUFFLE_KEY";
+
+ public static final String ORIGIN_MESSAGE_TRACE_ID = "origin_request_id";
+
+ /**
+ * 分片转发channel
+ */
+ protected transient ShuffleChannel shuffleChannel;
+
+ @Override
+ protected boolean batchInsert(List<IMessage> messageList) {
+ Map<Integer, JSONArray> shuffleMap = translateToShuffleMap(messageList);
+ if (shuffleMap != null && shuffleMap.size() > 0) {
+ Set<String> splitIds=new HashSet<>();
+ for (Map.Entry<Integer, JSONArray> entry : shuffleMap.entrySet()) {
+ ISplit split=shuffleChannel.getSplit(entry.getKey());
+ JSONObject msg=shuffleChannel.createMsg(entry.getValue(),split);
+ shuffleChannel.getProducer().batchAdd(new Message(msg),split);
+ splitIds.add(split.getQueueId());
+ }
+ shuffleChannel.getProducer().flush();
+ }
+ return true;
+ }
+
+ /**
+ * 对接收的消息按照不同shuffle key进行分组
+ *
+ * @param messages
+ * @return
+ */
+ protected Map<Integer, JSONArray> translateToShuffleMap(List<IMessage> messages) {
+ Map<Integer, JSONArray> shuffleMap = new HashMap<>();
+ for (IMessage msg : messages) {
+ if (msg.getHeader().isSystemMessage()) {
+ continue;
+ }
+ String shuffleKey = generateShuffleKey(msg);
+ if (StringUtil.isEmpty(shuffleKey)) {
+ shuffleKey="<null>";
+ LOG.debug("there is no group by value in message! " + msg.getMessageBody().toString());
+ //continue;
+ }
+ Integer index = shuffleChannel.hash(shuffleKey);
+ JSONObject body = msg.getMessageBody();
+ String offset = msg.getHeader().getOffset();
+ String queueId = msg.getHeader().getQueueId();
+
+ body.put(ORIGIN_OFFSET, offset);
+ body.put(ORIGIN_QUEUE_ID, queueId);
+ body.put(ORIGIN_MESSAGE_HEADER, JSONObject.toJSONString(msg.getHeader()));
+ body.put(ORIGIN_MESSAGE_TRACE_ID, msg.getHeader().getTraceId());
+ body.put(SHUFFLE_KEY, shuffleKey);
+
+ addPropertyToMessage(msg, body);
+
+ JSONArray jsonArray = shuffleMap.get(index);
+ if (jsonArray == null) {
+ jsonArray = new JSONArray();
+ shuffleMap.put(index, jsonArray);
+ }
+ jsonArray.add(body);
+
+ }
+ return shuffleMap;
+ }
+
+
+
+ /**
+ * 根据message生成shuffle key
+ *
+ * @param message
+ * @return
+ */
+ protected abstract String generateShuffleKey(IMessage message);
+
+ /**
+ * 如果需要额外的字段附加到shuffle前的message,通过实现这个子类增加
+ *
+ * @param oriJson
+ */
+ protected void addPropertyToMessage(IMessage oriMessage, JSONObject oriJson){
+
+ }
+
+ public ShuffleChannel getShuffleChannel() {
+ return shuffleChannel;
+ }
+
+ public void setShuffleChannel(ShuffleChannel shuffleChannel) {
+ this.shuffleChannel = shuffleChannel;
+ }
+}
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
new file mode 100644
index 0000000..3acefad
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java
@@ -0,0 +1,506 @@
+/*
+ * 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.model;
+
+import java.io.Serializable;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collection;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.common.utils.SQLUtil;
+import org.apache.rocketmq.streams.window.operator.AbstractWindow;
+import org.apache.rocketmq.streams.common.model.Entity;
+
+import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
+
+import org.apache.rocketmq.streams.common.context.IMessage;
+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.commons.lang3.tuple.Pair;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * 具体的窗口实例
+ */
+public class WindowInstance extends Entity implements Serializable {
+
+ protected static final Log LOG = LogFactory.getLog(WindowInstance.class);
+
+ private static final long serialVersionUID = 6893491128670330569L;
+
+ /**
+ * 窗口实例的开始时间
+ */
+ protected String startTime;
+
+ /**
+ * 窗口实例的结束时间
+ */
+ protected String endTime;
+
+ /**
+ * fire!
+ */
+ protected String fireTime;
+
+ /**
+ * 使用configName
+ */
+ protected String windowName;
+
+ protected String splitId;
+ /**
+ * namespace
+ */
+ protected String windowNameSpace;
+ protected String windowInstanceName;//默认等于窗口名,需要区分不同窗口时使用
+
+ /**
+ * splitId,windowNameSpace,windowName,windowInstanceName,windowInstanceName 数据库中存储的是MD5值
+ */
+ protected String windowInstanceSplitName;
+ /**
+ * windowInstanceId, splitId,windowNameSpace,windowName,windowInstanceName,windowInstanceName,startTime,endTime" 数据库中存储的是MD5值
+ */
+ protected String windowInstanceKey;
+
+ protected transient Boolean isNewWindowInstance = false;//当第一次创建时设置为true,否则设置为false
+
+ /**
+ * 0:待计算;1:已经计算结束;-1:已经取消;
+ */
+ protected int status = 0;
+
+ //todo 建议之后改个名字,为了测试方便,暂时用这个字段
+ protected Integer version = 1;//用于标识channel的状态,如果值是1,表示第一次消费,否则是第二次消费
+
+ private static SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+ private static final String SEPARATOR = "_";
+
+ protected transient Long lastMaxUpdateTime;//last max update time for session window
+
+ /**
+ * 创建window instance的唯一ID
+ *
+ * @return
+ */
+ public String createWindowInstanceId() {
+ return MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime);
+ }
+
+ /**
+ * 创建window instance对象列表
+ *
+ * @param window
+ * @param startAndEndTimeList
+ * @param fireTimeList
+ * @return
+ */
+ public static List<WindowInstance> createWindowInstances(AbstractWindow window,
+ List<Pair<String, String>> startAndEndTimeList, List<String> fireTimeList, String queueId) {
+ List<WindowInstance> lostInstanceList = new ArrayList<>();
+ for (int index = 0; index < startAndEndTimeList.size(); index++) {
+ Pair<String, String> pair = startAndEndTimeList.get(index);
+ WindowInstance windowInstance = window.createWindowInstance(pair.getLeft(), pair.getRight(), fireTimeList.get(index), queueId);
+
+ lostInstanceList.add(windowInstance);
+ }
+ return lostInstanceList;
+ }
+
+ public String createWindowInstancePartitionId() {
+ return StringUtil.createMD5Str(MapKeyUtil.createKey(windowNameSpace, windowName, windowInstanceName, startTime, endTime, splitId));
+ }
+ //
+ ///**
+ // * 批量查询窗口实例,对于不存在的窗口实例创建并插入数据库
+ // */
+ //public static List<WindowInstance> queryAndCreateWindowInstances(AbstractWindow window,
+ // List<Pair<String, String>> startAndEndTimeList, List<String> fireTimeList) {
+ // List<WindowInstance> windowInstanceList = queryWindowInstances(window, startAndEndTimeList, fireTimeList);
+ // if (windowInstanceList == null) {
+ // //TODO db exception
+ // return new ArrayList<>();
+ // }
+ // List<Pair<String, String>> lostPairList = new ArrayList<>();
+ // List<String> lostFireList = new ArrayList<>();
+ // for (int index = 0; index < startAndEndTimeList.size(); index++) {
+ // Pair<String, String> pair = startAndEndTimeList.get(index);
+ // String fireTime = fireTimeList.get(index);
+ // List<WindowInstance> resultList = windowInstanceList.stream().filter(
+ // instance -> instance.startTime.equals(pair.getLeft()) && instance.endTime.equals(pair.getRight()) && instance.getFireTime().equals(fireTime))
+ // .collect(
+ // Collectors.toList());
+ // if (CollectionUtil.isEmpty(resultList)) {
+ // lostPairList.add(pair);
+ // lostFireList.add(fireTimeList.get(index));
+ // }
+ // }
+ // List<WindowInstance> lostInstanceList = createWindowInstances(window,lostPairList,lostFireList);
+ // try {
+ // //插入数据库前重写windowInstanceId
+ //
+ // ORMUtil.batchIgnoreInto(lostInstanceList);
+ // } catch (Exception e) {
+ // LOG.error("failed in creating new window instances", e);
+ // }
+ // windowInstanceList.addAll(lostInstanceList);
+ // return windowInstanceList;
+ //}
+
+ ///**
+ // * 在内存中没有找到instance,查询数据库
+ // */
+ //public static WindowInstance queryWindowInstance(String dbWindowInstanceId) {
+ // String sql = "select * from " + ORMUtil.getTableName(WindowInstance.class)
+ // + " where status = 0 and window_instance_key = '" + dbWindowInstanceId + "';";
+ // WindowInstance windowInstance = ORMUtil.queryForObject(sql, null, WindowInstance.class);
+ // return windowInstance;
+ //}
+
+ //public static List<WindowInstance> queryWindowInstances(AbstractWindow window, List<Pair<String, String>> windowTimeList,
+ // List<String> fireTimeList) {
+ // if (CollectionUtil.isEmpty(windowTimeList)) {
+ // return new ArrayList<>();
+ // }
+ // String sql = "select * from " + ORMUtil.getTableName(WindowInstance.class);
+ // StringBuilder builder = new StringBuilder();
+ // builder.append(" where status = 0 and window_instance_key in (");
+ // for (int index = 0; index < windowTimeList.size(); index++) {
+ // Pair<String, String> windowTime = windowTimeList.get(index);
+ // String instanceKey = window.createWindowInstance(windowTime.getLeft(), windowTime.getRight(), fireTimeList.get(index)).getWindowInstanceKey();
+ // builder.append("'").append(instanceKey).append("'");
+ // if (index != windowTimeList.size() - 1) {
+ // builder.append(",");
+ // }
+ // }
+ // builder.append(");");
+ // sql += builder.toString();
+ // try {
+ // List<WindowInstance> dbWindowInstanceList = ORMUtil.queryForList(sql, null, WindowInstance.class);
+ // return dbWindowInstanceList;
+ // } catch (Exception e) {
+ // LOG.error("failed in getting window instances batch", e);
+ // }
+ // return null;
+ //}
+
+ /**
+ * 触发时间比lastTime小的所有的有效的instance
+ *
+ * @param
+ * @return
+ */
+ public static List<WindowInstance> queryAllWindowInstance(String lastTime, AbstractWindow window,
+ Collection<String> splitIds) {
+ if (window.isLocalStorageOnly() || splitIds == null) {
+ return null;
+ }
+ List<String> splitIdList = new ArrayList<>();
+ splitIdList.addAll(splitIds);
+ String[] splitNames = new String[splitIds.size()];
+ for (int i = 0; i < splitNames.length; i++) {
+ splitNames[i] = MapKeyUtil.createKey(window.getNameSpace(), window.getConfigureName(), splitIdList.get(i));
+ splitNames[i] = StringUtil.createMD5Str(splitNames[i]);
+ }
+ String sql = "select * from window_instance where "
+ + " status =0 and window_instance_split_name in(" + SQLUtil.createInSql(splitNames) + ")";
+
+ List<WindowInstance> dbWindowInstanceList = null;
+ try {
+ dbWindowInstanceList = ORMUtil.queryForList(sql, null, WindowInstance.class);
+ } catch (Exception e) {
+ LOG.error("failed in getting unfired window instances", e);
+ }
+ return dbWindowInstanceList;
+ }
+
+ /**
+ * 清理window
+ *
+ * @param windowInstance
+ */
+ public static void cleanWindow(WindowInstance windowInstance) {
+ List<WindowInstance> windowInstances = new ArrayList<>();
+ windowInstances.add(windowInstance);
+ clearInstances(windowInstances);
+ }
+
+ public static void clearInstances(List<WindowInstance> windowInstances) {
+ if (CollectionUtil.isEmpty(windowInstances)) {
+ return;
+ }
+ StringBuilder deleteInstanceSql = new StringBuilder();
+ boolean isFirst = true;
+ for (WindowInstance windowInstance : windowInstances) {
+ LOG.debug("clear window instance in db, instance key: " + windowInstance.getWindowInstanceKey());
+ if (isFirst) {
+ isFirst = false;
+ } else {
+ deleteInstanceSql.append(",");
+ }
+ deleteInstanceSql.append("('" + windowInstance.getWindowInstanceKey() + "')");
+ }
+ String deleteInstanceById = "delete from " + ORMUtil.getTableName(WindowInstance.class)
+ + " where (window_instance_key) in (" + deleteInstanceSql.toString() + ")";
+ ORMUtil.executeSQL(deleteInstanceById, null);
+ }
+
+ public static Long getOccurTime(AbstractWindow window, IMessage message) {
+ Long occurTime = null;
+ if (StringUtil.isEmpty(window.getTimeFieldName())) {
+ occurTime = message.getMessageBody().getLong("time");
+ if (occurTime == null) {
+ occurTime = message.getHeader().getSendTime();
+ }
+ } else {
+ try {
+ occurTime = message.getMessageBody().getLong(window.getTimeFieldName());
+ } catch (Exception e) {
+ String occurTimeString = message.getMessageBody().getString(window.getTimeFieldName());
+ try {
+ occurTime = dateFormat.parse(occurTimeString).getTime();
+ } catch (ParseException parseException) {
+ throw new RuntimeException("can not parse the time field (" + window.getTimeFieldName() + ")");
+ }
+ }
+ }
+ if (occurTime == null) {
+ throw new RuntimeException("can not parse the time field (" + window.getTimeFieldName() + ")");
+ }
+ return occurTime;
+ }
+
+ /**
+ * 查询或者创建Window的实例,滑动窗口有可能返回多个,滚动窗口返回一个
+ *
+ * @param window
+ * @param occurTime
+ * @return
+ * @Param isWindowInstance2DB 如果是秒级窗口,可能windowinstacne不必存表,只在内存保存,可以通过这个标志设置
+ */
+ public static List<WindowInstance> getOrCreateWindowInstance(AbstractWindow window, Long occurTime, int timeUnitAdjust, String queueId) {
+ int windowSlideInterval = window.getSlideInterval();
+ int windowSizeInterval = window.getSizeInterval();
+ if (windowSlideInterval == 0) {
+ windowSlideInterval = windowSizeInterval;
+ }
+ int waterMarkMinute = window.getWaterMarkMinute();
+ List<Date> windowBeginTimeList = DateUtil.getWindowBeginTime(occurTime, windowSlideInterval * timeUnitAdjust * 1000,
+ windowSizeInterval * timeUnitAdjust * 1000);
+ List<WindowInstance> instanceList = new ArrayList<>();
+ List<Pair<String, String>> lostWindowTimeList = new ArrayList<>();
+ List<String> lostFireList = new ArrayList<>();
+ long now = System.currentTimeMillis();
+ for (Date begin : windowBeginTimeList) {
+ Date end = DateUtil.addDate(TimeUnit.SECONDS, begin, windowSizeInterval * timeUnitAdjust);
+ Date fire = null;
+ if (window.getFireMode() != 0) {
+ //非正常触发模式
+ if (now - end.getTime() < 0) {
+ fire = end;
+ } else {
+ Long nowEventTime = window.getWindowMaxValueManager().updateWindowEventTime(queueId, begin.getTime());
+ List<Date> currentWindowList = DateUtil.getWindowBeginTime(
+ nowEventTime, windowSlideInterval * timeUnitAdjust * 1000,
+ windowSizeInterval * timeUnitAdjust * 1000);
+ if (!CollectionUtil.isEmpty(currentWindowList)) {
+ Date soonBegin = currentWindowList.get(currentWindowList.size() - 1);
+ Date soonEnd = DateUtil.addDate(TimeUnit.SECONDS, soonBegin,
+ windowSizeInterval * timeUnitAdjust);
+ Date soonFire = soonEnd;
+ fire = soonFire;
+ }
+ // System.out.println(DateUtil.format(fire));
+ if (fire.getTime() - end.getTime() - waterMarkMinute * timeUnitAdjust * 1000 > 0) {
+ //超过最大watermark,消息需要丢弃
+ break;
+ }
+ }
+ } else {
+ fire = DateUtil.addDate(TimeUnit.SECONDS, end, waterMarkMinute * timeUnitAdjust);
+ }
+ String startTime = DateUtil.format(begin);
+ String endTime = DateUtil.format(end);
+ String fireTime = DateUtil.format(fire);
+ String windowInstanceId = window.createWindowInstance(startTime, endTime, fireTime, queueId).createWindowInstanceId();
+ WindowInstance windowInstance = window.getWindowInstanceMap().get(windowInstanceId);
+ if (windowInstance == null) {
+ lostWindowTimeList.add(Pair.of(startTime, endTime));
+ lostFireList.add(fireTime);
+ } else {
+ windowInstance.setFireTime(fireTime);
+ instanceList.add(windowInstance);
+ }
+ }
+ List<WindowInstance> lostInstanceList = null;
+ lostInstanceList = WindowInstance.createWindowInstances(window, lostWindowTimeList, lostFireList, queueId);
+
+ instanceList.addAll(lostInstanceList);
+ for (WindowInstance windowInstance : instanceList) {
+ window.getWindowInstanceMap().putIfAbsent(windowInstance.createWindowInstanceId(), 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;
+ }
+
+ public void setStartTime(String startTime) {
+ this.startTime = startTime;
+ }
+
+ public String getEndTime() {
+ return endTime;
+ }
+
+ public void setEndTime(String endTime) {
+ this.endTime = endTime;
+ }
+
+ public String getFireTime() {
+ return fireTime;
+ }
+
+ public void setFireTime(String fireTime) {
+ this.fireTime = fireTime;
+ }
+
+ public String getWindowName() {
+ return windowName;
+ }
+
+ public void setWindowName(String windowName) {
+ this.windowName = windowName;
+ }
+
+ public String getWindowNameSpace() {
+ return windowNameSpace;
+ }
+
+ public void setWindowNameSpace(String windowNameSpace) {
+ this.windowNameSpace = windowNameSpace;
+ }
+
+ public int getStatus() {
+ return status;
+ }
+
+ public void setStatus(int status) {
+ this.status = status;
+ }
+
+ public Integer getVersion() {
+ return version;
+ }
+
+ public void setVersion(Integer version) {
+ this.version = version;
+ }
+
+ public String getWindowInstanceKey() {
+ return windowInstanceKey;
+ }
+
+ public String getWindowInstanceName() {
+ return windowInstanceName;
+ }
+
+ public void setWindowInstanceName(String windowInstanceName) {
+ this.windowInstanceName = windowInstanceName;
+ }
+
+ public void setWindowInstanceKey(String windowInstanceKey) {
+ this.windowInstanceKey = windowInstanceKey;
+ }
+
+ public Boolean isNewWindowInstance() {
+ return isNewWindowInstance;
+ }
+
+ public void setNewWindowInstance(Boolean newWindowInstance) {
+ isNewWindowInstance = newWindowInstance;
+ }
+
+ public String getSplitId() {
+ return splitId;
+ }
+
+ public void setSplitId(String splitId) {
+ this.splitId = splitId;
+ }
+
+ public String getWindowInstanceSplitName() {
+ return windowInstanceSplitName;
+ }
+
+ public void setWindowInstanceSplitName(String windowInstanceSplitName) {
+ this.windowInstanceSplitName = windowInstanceSplitName;
+ }
+
+ public Long getLastMaxUpdateTime() {
+ return lastMaxUpdateTime;
+ }
+
+ public void setLastMaxUpdateTime(Long lastMaxUpdateTime) {
+ this.lastMaxUpdateTime = lastMaxUpdateTime;
+ }
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/IWindowMaxValueManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/IWindowMaxValueManager.java
new file mode 100644
index 0000000..ae73327
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/IWindowMaxValueManager.java
@@ -0,0 +1,87 @@
+/*
+ * 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.offset;
+
+import org.apache.rocketmq.streams.window.model.WindowInstance;
+
+import java.util.Set;
+
+/**
+ * create split sequence number for window instance output result order by split sequence number when windown instance fire
+ */
+public interface IWindowMaxValueManager {
+
+ /**
+ * create split sequence number if the generator is not in memory, need load from db or other storage if instance is new ,set the split sequence number = init value
+ *
+ * @param instance
+ * @param splitId
+ * @return plus one on the current max split sequence number
+ */
+ Long incrementAndGetSplitNumber(WindowInstance instance, String splitId);
+
+ /**
+ * create split sequence number if the generator is not in memory, need load from db or other storage if instance is new ,set the split sequence number = init value
+ *
+ * @param key
+ * @return plus one on the current max split sequence number
+ */
+ Long incrementAndGetSplitNumber(String key);
+
+ /**
+ * return the max time from window receive event if (eventTime > maxEventTime) { maxEventTime=eventTime; } return maxEventTime
+ *
+ * @param eventTime
+ * @return
+ */
+ Long updateWindowEventTime(String splitId, Long eventTime);
+
+ //return the max time from window receive event
+ Long updateWindowEventTime(String splitId, String formatEventTime);
+
+ /**
+ * load mutil window instance split's max split num
+ *
+ * @param windowInstances
+ * @param split
+ */
+ void loadMaxSplitNum(Set<WindowInstance> windowInstances, String split);
+
+ String createSplitNumberKey(WindowInstance instance, String splitId);
+
+ /**
+ * load mutil window instance split's max split num
+ *
+ * @param keys
+ * @return
+ */
+ void loadMaxSplitNum(Set<String> keys);
+
+ void removeKeyPrefixFromLocalCache(Set<String> keyPrefixs);
+
+ //load window max event time
+ void loadWindowMaxEventTime(Set<String> splitId);
+
+ /**
+ * save addition WindowMaxValue
+ */
+ void flush();
+
+ void resetSplitNum(WindowInstance instance, String splitId);
+
+ void resetSplitNum(String key);
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValue.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValue.java
new file mode 100644
index 0000000..27b8b05
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValue.java
@@ -0,0 +1,89 @@
+/*
+ * 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.offset;
+
+import java.util.Date;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.rocketmq.streams.common.model.Entity;
+
+/**
+ * save windowintance max offset
+ */
+public class WindowMaxValue extends Entity {
+ public static long MAX_VALUE_BASE_VALUE=100000000;
+ protected String msgKey;
+ protected AtomicLong maxValue=new AtomicLong(MAX_VALUE_BASE_VALUE);
+
+ protected AtomicLong maxEventTime=new AtomicLong();//只有window需要
+
+ public WindowMaxValue(){
+ this.gmtModified=new Date();
+ this.gmtCreate=new Date();
+ }
+
+ public Long getMaxEventTime() {
+ return maxEventTime.get();
+ }
+
+ public void setMaxEventTime(Long maxEventTime) {
+ if(maxEventTime==null){
+ return;
+ }
+ this.maxEventTime.set( maxEventTime);
+ }
+
+ public String getMsgKey() {
+ return msgKey;
+ }
+
+ public void setMsgKey(String msgKey) {
+ this.msgKey = msgKey;
+ }
+
+ public Long getMaxValue() {
+ return maxValue.get();
+ }
+
+ public void setMaxValue(Long maxValue) {
+ this.maxValue.set(maxValue);
+ }
+
+ public long comareAndSet(Long eventTime){
+ if(eventTime==null){
+ return maxEventTime.get();
+ }
+ long old=maxEventTime.get();
+ if(old>=eventTime){
+ return old;
+ }
+ boolean updateSuccess=false;
+ while (!updateSuccess){
+ old=maxEventTime.get();
+ if(eventTime>old){
+ updateSuccess= maxEventTime.compareAndSet(old,eventTime);
+ } else {
+ break;
+ }
+ }
+ return maxEventTime.get();
+ }
+
+ public long incrementAndGetMaxOffset(){
+ return maxValue.incrementAndGet();
+ }
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueManager.java
new file mode 100644
index 0000000..b6d537c
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueManager.java
@@ -0,0 +1,247 @@
+/*
+ * 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.offset;
+
+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.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.orm.ORMUtil;
+import org.apache.rocketmq.streams.window.model.WindowInstance;
+import org.apache.rocketmq.streams.window.operator.AbstractWindow;
+
+import static org.apache.rocketmq.streams.window.offset.WindowMaxValue.MAX_VALUE_BASE_VALUE;
+
+public class WindowMaxValueManager implements IWindowMaxValueManager {
+ protected AbstractWindow window;
+
+ protected Map<String, WindowMaxValue> windowOffsetMap=new HashMap<>();//all window offsets
+ protected List<WindowMaxValue> needUpdateWindowValues =new ArrayList<>();//new windowoffset list, need save to storage when flush
+
+ public WindowMaxValueManager(AbstractWindow window){
+ this.window=window;
+ }
+
+ @Override
+ public String createSplitNumberKey(WindowInstance instance, String splitId){
+ String key= MapKeyUtil.createKey(splitId,instance.getWindowInstanceKey());
+ return key;
+ }
+
+ @Override
+ public Long incrementAndGetSplitNumber(WindowInstance instance, String splitId) {
+ String key=createSplitNumberKey(instance,splitId);
+ WindowMaxValue windowMaxValue=queryOrCreateWindowOffset(key,true);
+ return windowMaxValue.incrementAndGetMaxOffset();
+ }
+
+ @Override
+ public Long updateWindowEventTime(String splitId, Long eventTime) {
+ String windowId=StringUtil.createMD5Str(MapKeyUtil.createKey(window.getNameSpace(),window.getConfigureName()));
+ String key=MapKeyUtil.createKey(splitId, windowId);
+ WindowMaxValue windowOffset=queryOrCreateWindowOffset(key,true);
+ return windowOffset.comareAndSet(eventTime);
+ }
+
+ @Override
+ public Long updateWindowEventTime(String splitId, String formatEventTime) {
+ if(StringUtil.isEmpty(formatEventTime)){
+ return updateWindowEventTime(splitId,(Long)null);
+ }
+ Long time= DateUtil.parseTime(formatEventTime).getTime();
+ return updateWindowEventTime(splitId,time);
+ }
+
+ @Override
+ public Long incrementAndGetSplitNumber(String key) {
+ WindowMaxValue windowOffset=queryOrCreateWindowOffset(key,true);
+ return windowOffset.incrementAndGetMaxOffset();
+ }
+
+ @Override
+ public void loadMaxSplitNum(Set<WindowInstance> windowInstances, String splitId) {
+ if(windowInstances==null||StringUtil.isEmpty(splitId)){
+ return;
+ }
+ Set<String> keys=new HashSet<>();
+ for(WindowInstance instance:windowInstances){
+ String key= createSplitNumberKey(instance,splitId);
+ keys.add(key);
+ }
+ queryOrCreateWindowOffset(keys,window.isLocalStorageOnly());
+ }
+
+ @Override
+ public void loadMaxSplitNum(Set<String> keys) {
+ queryOrCreateWindowOffset(keys,window.isLocalStorageOnly());
+ }
+
+ @Override
+ public void loadWindowMaxEventTime(Set<String> splitIds) {
+ if(splitIds==null){
+ return;
+ }
+ Set<String> keys=new HashSet<>();
+ for(String splitId:splitIds){
+ String windowId=StringUtil.createMD5Str(MapKeyUtil.createKey(window.getNameSpace(),window.getConfigureName()));
+ String key=MapKeyUtil.createKey(splitId, windowId);
+ keys.add(key);
+ }
+
+ queryOrCreateWindowOffset(keys,window.isLocalStorageOnly());
+ }
+
+ @Override
+ public void flush(){
+ if(window.isLocalStorageOnly()){
+ needUpdateWindowValues =new ArrayList<>();
+ return;
+ }
+ List<WindowMaxValue> windowOffsetList=new ArrayList<>();
+ synchronized (this){
+ windowOffsetList.addAll(needUpdateWindowValues);
+ needUpdateWindowValues =new ArrayList<>();
+ }
+ ORMUtil.batchReplaceInto(windowOffsetList);
+ }
+
+ @Override
+ public void resetSplitNum(WindowInstance instance, String splitId) {
+ String key=createSplitNumberKey(instance,splitId);
+ resetSplitNum(key);
+ }
+
+ @Override
+ public synchronized void resetSplitNum(String key) {
+ WindowMaxValue windowMaxValue=queryOrCreateWindowOffset(key,window.isLocalStorageOnly());
+ windowMaxValue.maxValue.set(MAX_VALUE_BASE_VALUE);
+ needUpdateWindowValues.add(windowMaxValue);
+ }
+
+ @Override
+ public synchronized void removeKeyPrefixFromLocalCache(Set<String> keyPrefixs) {
+ Map<String, WindowMaxValue> copy=new HashMap<>();
+ copy.putAll(this.windowOffsetMap);
+ for(String key:copy.keySet()){
+ for(String keyPrefix:keyPrefixs){
+ if(key.startsWith(keyPrefix)){
+ this.windowOffsetMap.remove(key);
+ }
+ }
+ }
+
+ }
+
+ /**
+ * 查询window的总计数器
+ *
+ * @return
+ */
+ protected WindowMaxValue queryOrCreateWindowOffset(String key,boolean onlyLocal){
+ Set<String> keys=new HashSet<>();
+ keys.add(key);
+ Map<String,WindowMaxValue> windowMaxValueMap=queryOrCreateWindowOffset(keys,onlyLocal);
+ if(windowMaxValueMap==null){
+ return null;
+ }
+ return windowMaxValueMap.values().iterator().next();
+ }
+
+ /**
+ * 查询window的总计数器
+ *
+ * @return
+ */
+ protected Map<String,WindowMaxValue> queryOrCreateWindowOffset(Set<String> keys,boolean onlyLocal){
+ Map<String,WindowMaxValue> result=new HashMap<>();
+ if(keys==null){
+ return result;
+ }
+ List<String> keyNotInLocal=new ArrayList<>();
+ for(String key:keys){
+ WindowMaxValue windowMaxValue=windowOffsetMap.get(key);
+ if(windowMaxValue !=null){
+ result.put(key,windowMaxValue);
+ }else if(onlyLocal){
+ windowMaxValue=create(key);
+ result.put(key,windowMaxValue);
+ }else {
+ keyNotInLocal.add(key);
+ }
+ }
+
+ if(onlyLocal){
+ return result;
+ }
+ if(keyNotInLocal.size()==0){
+ return result;
+ }
+ synchronized (this){
+ List<String> synchKeyNotInLocal=new ArrayList<>();
+ for(String key:keyNotInLocal) {
+ WindowMaxValue windowMaxValue = windowOffsetMap.get(key);
+ if (windowMaxValue != null) {
+ result.put(key, windowMaxValue);
+ }else {
+ synchKeyNotInLocal.add(key);
+ }
+ }
+ List<WindowMaxValue> windowMaxValues=null;
+ if(synchKeyNotInLocal.size()>0){
+ String sql="select * from "+ ORMUtil.getTableName(WindowMaxValue.class)+ " where msg_key in ("+ SQLUtil.createInSql(synchKeyNotInLocal) +")";
+ windowMaxValues=ORMUtil.queryForList(sql,null, WindowMaxValue.class);
+
+ }
+ // String key= MapKeyUtil.createKey(window.getNameSpace(),window.getConfigureName(),split);
+ List<String> keysNotInDB=new ArrayList<>();
+ keysNotInDB.addAll(synchKeyNotInLocal);
+ if(windowMaxValues!=null){
+ for(WindowMaxValue windowMaxValue:windowMaxValues){
+ result.put(windowMaxValue.getMsgKey(), windowMaxValue);
+ keysNotInDB.remove(windowMaxValue);
+ windowOffsetMap.put(windowMaxValue.getMsgKey(),windowMaxValue);
+ }
+ }
+ if(keysNotInDB!=null&&keysNotInDB.size()>0){
+ for(String key:keysNotInDB){
+ result.put(key, create(key));
+ }
+ }
+ }
+ return result;
+ }
+
+ protected WindowMaxValue create(String key){
+ WindowMaxValue windowMaxValue=new WindowMaxValue();
+ windowMaxValue.setGmtCreate(new Date());
+ windowMaxValue.setGmtModified(new Date());
+ windowMaxValue.setMsgKey(key);
+ windowMaxValue.setMaxValue(MAX_VALUE_BASE_VALUE);
+ windowMaxValue.setMaxEventTime(null);
+ needUpdateWindowValues.add(windowMaxValue);
+ windowOffsetMap.put(key,windowMaxValue);
+ return windowMaxValue;
+ }
+}
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
new file mode 100644
index 0000000..cb20f75
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import org.apache.rocketmq.streams.common.configurable.IAfterConfiguableRefreshListerner;
+import org.apache.rocketmq.streams.common.configurable.IConfigurableService;
+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.storage.WindowStorage;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public abstract class AbstractShuffleWindow extends AbstractWindow implements IAfterConfiguableRefreshListerner {
+
+ protected transient ShuffleChannel shuffleChannel;
+ protected transient AtomicBoolean hasCreated = new AtomicBoolean(false);
+
+ @Override
+ protected boolean initConfigurable() {
+ storage = new WindowStorage();
+ storage.setLocalStorageOnly(isLocalStorageOnly);
+ return super.initConfigurable();
+ }
+
+ @Override
+ public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) {
+ if (hasCreated.compareAndSet(false, true)) {
+ this.windowFireSource = new WindowRireSource(this);
+ this.windowFireSource.init();
+ this.windowFireSource.start(getFireReceiver());
+ this.shuffleChannel = new ShuffleChannel(this);
+ windowCache.setBatchSize(5000);
+ windowCache.setShuffleChannel(shuffleChannel);
+ shuffleChannel.startChannel();
+ }
+ }
+
+ @Override
+ public int fireWindowInstance(WindowInstance windowInstance, Map<String, String> queueId2Offset) {
+ Set<String> splitIds = new HashSet<>();
+ splitIds.add(windowInstance.getSplitId());
+ shuffleChannel.flush(splitIds);
+ int fireCount = fireWindowInstance(windowInstance, windowInstance.getSplitId(), queueId2Offset);
+ return fireCount;
+ }
+
+ /**
+ * 接收shuffle后的消息进行计算,子类实现具体计算逻辑
+ *
+ * @param messages
+ * @param instance
+ */
+ public abstract void shuffleCalculate(List<IMessage> messages, WindowInstance instance, String queueId);
+
+ /**
+ * 触发window
+ *
+ * @param instance
+ */
+ protected abstract int fireWindowInstance(WindowInstance instance, String queueId, Map<String, String> queueId2Offset);
+
+ public abstract void clearCache(String queueId);
+}
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
new file mode 100644
index 0000000..6d23157
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java
@@ -0,0 +1,781 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.apache.rocketmq.streams.common.topology.ChainStage.PiplineRecieverAfterCurrentNode;
+import org.apache.rocketmq.streams.common.topology.stages.udf.IReducer;
+import org.apache.rocketmq.streams.common.topology.stages.udf.IRedurce;
+import org.apache.rocketmq.streams.common.utils.Base64Utils;
+import org.apache.rocketmq.streams.common.utils.InstantiationUtil;
+import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
+import org.apache.rocketmq.streams.script.utils.FunctionUtils;
+import org.apache.rocketmq.streams.window.model.FunctionExecutor;
+import org.apache.rocketmq.streams.window.model.WindowInstance;
+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.state.impl.WindowValue;
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.MessageHeader;
+import org.apache.rocketmq.streams.common.topology.ChainStage;
+import org.apache.rocketmq.streams.common.topology.builder.IStageBuilder;
+import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder;
+import org.apache.rocketmq.streams.common.topology.stages.WindowChainStage;
+import org.apache.rocketmq.streams.common.topology.model.IWindow;
+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.script.parser.imp.FunctionParser;
+import org.apache.rocketmq.streams.script.operator.impl.AggregationScript;
+import org.apache.rocketmq.streams.script.operator.impl.FunctionScript;
+import org.apache.rocketmq.streams.script.service.IScriptExpression;
+import org.apache.rocketmq.streams.script.service.IScriptParamter;
+import org.apache.rocketmq.streams.script.operator.expression.ScriptExpression;
+import org.apache.rocketmq.streams.script.service.IAccumulator;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.window.storage.WindowStorage;
+
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+
+/**
+ * window definition in the pipeline, created by user's configure in WindowChainStage
+ */
+public abstract class AbstractWindow extends BasedConfigurable implements IWindow, IStageBuilder<ChainStage> {
+
+ protected static final Log LOG = LogFactory.getLog(AbstractWindow.class);
+
+ /**
+ * tumble or hop window 目前不再使用了
+ */
+ protected String windowType;
+
+ /**
+ * 用消息中的哪个字段做时间字段
+ */
+ protected String timeFieldName ;
+
+ /**
+ * having column in having clause eg: key:'having_sum_0001' value:'having_sum_0001=SUM(OrderPrice)<2000' note: here ignore the logical relation value may be multi expression which split by ${SCRIPT_SPLIT_CHAR} update: change sql(move the function into select clause) to escape function in having clause
+ */
+ private Map<String, String> havingMap = new HashMap<>(16);
+
+ /**
+ * computed column in select clause eg: key:'max_valid_user_fail_host_cnt' value:'max_valid_user_fail_host_cnt=max(base_line_invalid_user_fail_host_cnt)' note: 1) value may be multi expression which split by ${SCRIPT_SPLIT_CHAR} 2) computed column can't be same
+ */
+ private Map<String, String> selectMap = new HashMap<>(16);
+
+ /**
+ * SQL中group by的字段,使用;拼接,如"name;age"
+ */
+ protected String groupByFieldName ;
+
+ /**
+ * 意义同blink中,允许最晚的消息到达时间,单位是分钟
+ */
+ protected int waterMarkMinute = 0;
+
+ /**
+ * size or step of window, unit: minute
+ */
+ protected int sizeInterval;
+ /**
+ * the period of hop window, unit: minute
+ */
+ protected int slideInterval;
+ /**
+ * 主要是做兼容,以前设计的窗口时间是分钟为单位,如果有秒作为窗口时间的,通过设置timeUntiAdjust=1来实现。 后续需要调整成直接秒级窗口
+ */
+ protected int timeUnitAdjust=60;
+ /**
+ * the variable name of window size which can be got from message
+ */
+ protected String sizeVariable;
+
+ /**
+ * the coefficient to adjust window size for use minute as unit
+ */
+ protected Integer sizeAdjust;
+
+ /**
+ * the coefficient to adjust window slide for use minute as unit
+ */
+ protected Integer slideAdjust;
+
+ /**
+ * the variable name of window slide which it's value can be got from message
+ */
+ protected String slideVariable;
+
+ /**
+ * 默认为空,窗口的触发类似flink,在测试模式下,因为消息有界,期望当消息发送完成后能触发,可以设置两条消息的最大间隔,超过这个间隔,将直接触发消息
+ */
+ protected Long msgMaxGapSecond;
+
+ /**
+ * 是否支持过期数据的计算 过期:当前时间大于数据所在窗口的触发时间
+ */
+ protected int fireMode=0;//0:普通触发,firetime后收到数据丢弃;1:多实例多次独立触发,在watermark时间内,同starttime,endtime创建多个实例,多次触发;2.单实例,多次独立触发,每次触发是最新值
+
+ protected boolean isLocalStorageOnly=false;//是否只用本地存储,可以提高性能,但不保证可靠性
+ protected String reduceSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码
+ protected transient IReducer reducer;
+ /**
+ * the computed column and it's process of computing
+ */
+ private transient Map<String, List<FunctionExecutor>> columnExecuteMap = new HashMap<>(16);
+
+ /**
+ * used in last part to filter and transfer field in case data lost during firing
+ */
+ protected transient Map<String, String> columnProjectMap = new HashMap<>();
+
+ /**
+ * 当前计算节点的PipeLine里的Window实例对象,方便基于时间快速定位 key:namespace;configName(这里理解成windowName);startTime;endTime value:WindowInstance
+ */
+ protected transient ConcurrentHashMap<String, WindowInstance> windowInstanceMap = new ConcurrentHashMap<>();
+
+ /**
+ * 触发窗口后需要执行的逻辑
+ */
+ protected transient PiplineRecieverAfterCurrentNode fireReceiver;
+
+ /**
+ * 全局名称
+ */
+ protected transient String WINDOW_NAME;
+
+ /**
+ * 内部使用,定期检查窗口有没有触发
+ */
+ //protected transient ScheduledExecutorService fireWindowInstanceChecker =new ScheduledThreadPoolExecutor(3);
+
+ // protected transient ExecutorService deleteService = Executors.newSingleThreadExecutor();
+
+ protected volatile transient WindowCache windowCache;
+ protected transient WindowStorage storage;
+ protected transient WindowRireSource windowFireSource;
+
+ //create and save window instacne max partitionNum and window max eventTime
+ protected transient IWindowMaxValueManager windowMaxValueManager;
+
+ public AbstractWindow() {
+ setType(IWindow.TYPE);
+ }
+
+ @Override
+ protected boolean initConfigurable() {
+ boolean success = super.initConfigurable();
+ /**
+ * 如果没有db配置,不开启远程存储服务
+ */
+ if(!ORMUtil.hasConfigueDB()){
+ isLocalStorageOnly=true;
+ }
+ AbstractWindow window=this;
+ windowCache=new WindowCache(){
+
+ @Override
+ protected String generateShuffleKey(IMessage message) {
+ return window.generateShuffleKey(message);
+ }
+ };
+ windowCache.init();
+ windowCache.openAutoFlush();
+
+ WINDOW_NAME = MapKeyUtil.createKey(getNameSpace(), getConfigureName());
+ //fireDelaySecond时间检查一次是否有窗口需要触发
+ //fireWindowInstanceChecker.scheduleWithFixedDelay(this, 0, 5, TimeUnit.SECONDS);
+ initFunctionExecutor();
+ //启动shuffle channel 实现消息shuffle以及接收shuffle消息并处理
+ // FireManager.getInstance().startFireCheck();
+ if(StringUtil.isNotEmpty(this.reduceSerializeValue)){
+ byte[] bytes= Base64Utils.decode( this.reduceSerializeValue);
+ reducer = InstantiationUtil.deserializeObject(bytes);
+ }
+ windowMaxValueManager = new WindowMaxValueManager(this);
+ return success;
+ }
+
+ /**
+ * 对于一条消息来说,window 首先需要检查是否有窗口实例,如果没有则创建。如果窗口实例已经超过最大的water mark,数据丢弃,否则进行消息积累 消息会先经历batchAdd 然后flush
+ *
+ * @param message
+ * @param context
+ * @return
+ */
+ @Override
+ public AbstractContext<IMessage> doMessage(IMessage message, AbstractContext context) {
+ if (StringUtils.isNotEmpty(sizeVariable)) {
+ if (message.getMessageBody().containsKey(sizeVariable)) {
+ try {
+ this.sizeInterval = sizeAdjust * message.getMessageBody().getInteger(sizeVariable);
+ } catch (Exception e) {
+ LOG.error("failed in getting the size value, message = " + message.toString(), e);
+ }
+ }
+ }
+ if (StringUtils.isNotEmpty(slideVariable)) {
+ if (message.getMessageBody().containsKey(slideVariable)) {
+ try {
+ this.slideInterval = slideAdjust * message.getMessageBody().getInteger(slideVariable);
+ } catch (Exception e) {
+ LOG.error("failed in getting the slide value, message = " + message.toString(), e);
+ }
+ }
+ }
+ // List<WindowInstance> windowInstanceList = queryOrCreateWindowInstance(message);
+ JSONObject msg = message.getMessageBody();
+ msg.put(MessageHeader.class.getSimpleName(), message.getHeader());
+ msg.put(AbstractWindow.class.getSimpleName(), this);
+ windowCache.batchAdd(message);
+ //主要为了在单元测试中,写入和触发一体化使用,无实际意义,不要在业务场景使用这个字段
+
+ // TraceUtil.debug(message.getHeader().getTraceId(), "origin message in", message.getMessageBody().toJSONString());
+ return context;
+
+ }
+
+ /*
+
+ public String createWindowInstance(String startTime,String endTime,String fireTime){
+
+ }
+ */
+
+ public WindowInstance createWindowInstance(String startTime, String endTime, String fireTime,String splitId) {
+ WindowInstance windowInstance =new WindowInstance();
+ windowInstance.setFireTime(fireTime);
+ windowInstance.setStartTime(startTime);
+ windowInstance.setEndTime(endTime);
+ windowInstance.setSplitId(splitId);
+ windowInstance.setGmtCreate(new Date());
+ windowInstance.setGmtModified(new Date());
+ windowInstance.setWindowInstanceName(createWindowInstanceName(startTime,endTime,fireTime));
+ windowInstance.setWindowName(getConfigureName());
+ windowInstance.setWindowNameSpace(getNameSpace());
+ String windowInstanceId =windowInstance.createWindowInstanceId();
+ String dbWindowInstanceId = StringUtil.createMD5Str(windowInstanceId);
+ windowInstance.setWindowInstanceKey(dbWindowInstanceId);
+
+ windowInstance.setWindowInstanceSplitName(StringUtil.createMD5Str(MapKeyUtil.createKey(getNameSpace(), getConfigureName(),splitId)));
+ windowInstance.setNewWindowInstance(true);
+ return windowInstance;
+ }
+
+ /**
+ * 创建window instance name
+ *
+ * @param startTime
+ * @param endTime
+ * @param fireTime
+ * @return
+ */
+ public String createWindowInstanceName(String startTime, String endTime, String fireTime){
+ return fireMode==1?fireTime:getConfigureName();
+ }
+
+ /**
+ * 获取这个窗口实例,这个分片最大的序列号,如果是新窗口,从1开始
+ *
+ * @param instance
+ * @param shuffleId
+ * @return
+ */
+
+ public long incrementAndGetSplitNumber(WindowInstance instance,String shuffleId){
+ long maxValue= windowMaxValueManager.incrementAndGetSplitNumber(instance,shuffleId);
+ return maxValue;
+ }
+
+ public abstract Class getWindowBaseValueClass();
+
+ public abstract int fireWindowInstance(WindowInstance windowInstance,Map<String,String>queueId2Offset) ;
+
+ /**
+ * 计算每条记录的group by值,对于groupby分组,里面任何字段不能为null值,如果为null值,这条记录会被忽略
+ *
+ * @param message
+ * @return
+ */
+ protected String generateShuffleKey(IMessage message){
+ if (StringUtil.isEmpty(groupByFieldName)) {
+ return null;
+ }
+ JSONObject msg=message.getMessageBody();
+ String[] fieldNames = groupByFieldName.split(";");
+ String[] values=new String[fieldNames.length];
+ boolean isFirst = true;
+ int i=0;
+ for (String filedName : fieldNames) {
+ if (isFirst) {
+ isFirst = false;
+ }
+ String value = msg.getString(filedName);
+ values[i]=value;
+ i++;
+ }
+ return MapKeyUtil.createKey(values);
+ }
+
+ public abstract void clearFire(List<WindowInstance> windowInstances);
+
+ public void clearFire(WindowInstance windowInstance){
+ if(windowInstance==null){
+ return;
+ }
+ List<WindowInstance>windowInstances=new ArrayList<>();
+ windowInstances.add(windowInstance);
+ clearFire(windowInstances);
+ }
+
+ /**
+ * init the function executor TODO: 1) function executor may be parsed in parser module;
+ */
+ protected void initFunctionExecutor() {
+ //
+ columnExecuteMap.clear();
+ columnProjectMap.clear();
+ //
+ for (Entry<String, String> entry : selectMap.entrySet()) {
+ String computedColumn = entry.getKey();
+ columnProjectMap.put(computedColumn, computedColumn);
+ String scriptString = entry.getValue();
+ if (StringUtil.isEmpty(computedColumn) || StringUtil.isEmpty(scriptString)) {
+ LOG.warn(
+ "computed column or it's expression can not be empty! column = " + computedColumn + " expression = "
+ + scriptString);
+ continue;
+ }
+ if (computedColumn.equals(scriptString)) {
+ continue;
+ }
+ //
+ LinkedList<FunctionExecutor> scriptExecutorList = new LinkedList<>();
+ List<IScriptExpression> functionList = new ArrayList<>();
+ try {
+ functionList = FunctionParser.getInstance().parse(scriptString);
+ } catch (Exception e) {
+ LOG.error("failed in parsing script expression = " + scriptString + " window = " + WINDOW_NAME);
+ throw new RuntimeException("failed in parsing operator expression = " + scriptString);
+ }
+ if (CollectionUtil.isNotEmpty(functionList)) {
+ StringBuilder scriptBuilder = new StringBuilder();
+ for (IScriptExpression expression : functionList) {
+ String functionName = expression.getFunctionName();
+ List<IScriptParamter> scriptParameterList = expression.getScriptParamters();
+ String theScript = expression.getExpressionDescription();
+ IAccumulator director = AggregationScript.getAggregationFunction(functionName);
+ if (director != null) {
+ if (scriptBuilder.length() != 0) {
+ FunctionScript scalarEngine = new FunctionScript(scriptBuilder.toString());
+ scalarEngine.init();
+ scriptExecutorList.add(
+ new FunctionExecutor(computedColumn + "_" + scriptExecutorList.size(), scalarEngine));
+ scriptBuilder = new StringBuilder();
+ }
+ String[] functionParameterNames = scriptParameterList.stream().map(
+ scriptParameter -> scriptParameter.getScriptParameterStr()).collect(Collectors.toList())
+ .toArray(new String[0]);
+ AggregationScript accEngine = new AggregationScript(
+ ((ScriptExpression)expression).getNewFieldName(), functionName,
+ functionParameterNames);
+ accEngine.setDirector(director);
+ scriptExecutorList.add(
+ new FunctionExecutor(computedColumn + "_" + scriptExecutorList.size(), accEngine));
+ } else {
+ scriptBuilder.append(theScript).append(SCRIPT_SPLIT_CHAR);
+ }
+ }
+ if (scriptBuilder.length() != 0) {
+ FunctionScript scalarEngine = new FunctionScript(scriptBuilder.toString());
+ scalarEngine.init();
+ scriptExecutorList.add(
+ new FunctionExecutor(computedColumn + "_" + scriptExecutorList.size(), scalarEngine));
+ }
+ columnExecuteMap.put(computedColumn, scriptExecutorList);
+ } else {
+ LOG.error("parser's result is empty, script expression = " + scriptString + " window = " + WINDOW_NAME);
+ throw new RuntimeException("parser's result is empty, operator expression = " + scriptString);
+ }
+ }
+ if (LOG.isDebugEnabled()) {
+ Iterator<Entry<String, List<FunctionExecutor>>> iterator = columnExecuteMap.entrySet().iterator();
+ LOG.debug("window function execute split as follows:\t");
+ while (iterator.hasNext()) {
+ Entry<String, List<FunctionExecutor>> entry = iterator.next();
+ StringBuilder builder = new StringBuilder();
+ for (FunctionExecutor executor : entry.getValue()) {
+ if (executor.getExecutor() instanceof AggregationScript) {
+ builder.append(((AggregationScript)executor.getExecutor()).getFunctionName()).append("\t");
+ } else {
+ builder.append(((FunctionScript)executor.getExecutor()).getScript()).append("\t");
+ }
+ }
+ LOG.debug(entry.getKey() + " -> " + builder.toString());
+ }
+ }
+ }
+
+ /**
+ * 根据消息获取对应的window instance 列表
+ *
+ * @param message
+ * @return
+ */
+ public List<WindowInstance> queryOrCreateWindowInstance(IMessage message,String queueId) {
+ List<WindowInstance> windowInstances=WindowInstance.getOrCreateWindowInstance(this, WindowInstance.getOccurTime(this, message), timeUnitAdjust,
+ queueId);
+ if(fireMode==2){
+ if(windowInstances==null){
+ return null;
+ }
+ for(WindowInstance windowInstance:windowInstances){
+ Date endTime=DateUtil.parseTime(windowInstance.getEndTime());
+ Date lastFireTimne=DateUtil.addDate(TimeUnit.SECONDS,endTime,getWaterMarkMinute()*timeUnitAdjust);
+ //if fireMode==2, need clear data in lastFireTime
+ WindowInstance lastClearWindowInstance=createWindowInstance(windowInstance.getStartTime(),windowInstance.getEndTime(),DateUtil.format(lastFireTimne),queueId);
+ getWindowFireSource().registFireWindowInstanceIfNotExist(lastClearWindowInstance,this);
+ }
+ }
+ return windowInstances;
+ }
+
+ /**
+ * 获取实际触发的时间,增加了固定延迟和water marker
+ *
+ * @param windowInstance
+ * @return
+ */
+ public Date getRealFireTime(WindowInstance windowInstance){
+ Date fireDate=DateUtil.parse(windowInstance.getFireTime());
+ return DateUtil.addSecond(fireDate,FIRE_DELAY_SECOND);
+ }
+
+ /**
+ * 获取window处理的消息中最大的时间
+ * @param msg
+ * @return
+ */
+ public Long updateMaxEventTime(IMessage msg){
+ String splitId=msg.getHeader().getQueueId();
+ if(StringUtil.isEmpty(this.timeFieldName)){
+ return updateWindowEventTime(splitId,System.currentTimeMillis());
+ }
+ String time=msg.getMessageBody().getString(timeFieldName);
+ if(StringUtil.isEmpty(time)){
+ LOG.error(timeFieldName+ " is null, may be delay window fire");
+ return null;
+ }
+ if(FunctionUtils.isLong(time)){
+ return updateWindowEventTime(splitId,Long.valueOf(time));
+ }
+ return updateWindowEventTime(splitId,time);
+ }
+
+ protected transient Map<String,Long> maxTime=new HashMap<>();
+ private Long updateWindowEventTime(String splitId, String currentTime) {
+ return updateWindowEventTime(splitId,DateUtil.parseTime(currentTime).getTime());
+ }
+ private Long updateWindowEventTime(String splitId, Long currentTime) {
+ Long old=maxTime.get(splitId);
+
+ if(old==null||currentTime>old){
+ maxTime.put(splitId,currentTime);
+ }
+ return maxTime.get(splitId);
+ }
+
+
+ /**
+ * 聚合后的数据,继续走规则引擎的规则
+ *
+ * @param windowValueList
+ */
+ public void sendFireMessage(List<WindowValue> windowValueList,String queueId) {
+ int count = 0;
+ for (WindowValue windowValue : windowValueList) {
+ JSONObject message = new JSONObject();
+
+ if(JSONObject.class.isInstance(windowValue.getcomputedResult())){
+ message=(JSONObject)windowValue.getcomputedResult();
+ }else {
+ Iterator<Entry<String, Object>> it = windowValue.iteratorComputedColumnResult();
+ while (it.hasNext()) {
+ Entry<String, Object> entry = it.next();
+ message.put(entry.getKey(), entry.getValue());
+ }
+ }
+
+ Long fireTime=DateUtil.parseTime(windowValue.getFireTime()).getTime();
+ long baseTime= 1577808000000L ;//set base time from 2021-01-01 00:00:00
+ int sameFireCount=0;
+ if(fireMode!=0){
+ Long endTime=DateUtil.parseTime(windowValue.getEndTime()).getTime();
+ sameFireCount=(int)((fireTime-endTime)/1000)/sizeInterval*timeUnitAdjust;
+ if(sameFireCount>=1){
+ sameFireCount=1;
+ }
+ }
+ //can keep offset in order
+ Long offset=((fireTime-baseTime)/1000*10+sameFireCount)*100000000+windowValue.getPartitionNum();
+ message.put("windowInstanceId",windowValue.getWindowInstancePartitionId());
+ message.put("offset",offset);
+ Message newMessage=windowFireSource.createMessage(message,queueId,offset+"",false);
+ newMessage.getHeader().setOffsetIsLong(true);
+ if (count == windowValueList.size() - 1) {
+ newMessage.getHeader().setNeedFlush(true);
+ }
+ windowFireSource.executeMessage(newMessage);
+ count++;
+ }
+ }
+
+ @Override
+ public ChainStage createStageChain(PipelineBuilder pipelineBuilder) {
+ pipelineBuilder.addConfigurables(this);
+ WindowChainStage windowChainStage = new WindowChainStage();
+ windowChainStage.setWindow(this);
+ windowChainStage.setNameSpace(getNameSpace());
+ return windowChainStage;
+ }
+
+ @Override
+ public void addConfigurables(PipelineBuilder pipelineBuilder) {
+
+ }
+
+ public void setSizeVariable(String variableName) {
+ sizeVariable = variableName;
+ }
+
+ public String getWindowType() {
+ return windowType;
+ }
+
+ public void setWindowType(String windowType) {
+ this.windowType = windowType;
+ }
+
+ public String getTimeFieldName() {
+ return timeFieldName;
+ }
+
+ public void setTimeFieldName(String timeFieldName) {
+ this.timeFieldName = timeFieldName;
+ }
+
+ public Map<String, String> getSelectMap() {
+ return selectMap;
+ }
+
+ public void setSelectMap(Map<String, String> selectMap) {
+ this.selectMap = selectMap;
+ }
+
+ public Map<String, List<FunctionExecutor>> getColumnExecuteMap() {
+ return columnExecuteMap;
+ }
+
+ public Map<String, String> getColumnProjectMap() {
+ return columnProjectMap;
+ }
+
+ public String getGroupByFieldName() {
+ return groupByFieldName;
+ }
+
+ public void setGroupByFieldName(String groupByFieldName) {
+ this.groupByFieldName = groupByFieldName;
+ }
+
+ public int getWaterMarkMinute() {
+ return waterMarkMinute;
+ }
+
+ public void setWaterMarkMinute(int waterMarkMinute) {
+ this.waterMarkMinute = waterMarkMinute;
+ }
+
+ public int getSizeInterval() {
+ return sizeInterval;
+ }
+
+ public void setSizeInterval(int sizeInterval) {
+ this.sizeInterval = sizeInterval;
+ }
+
+ public ConcurrentHashMap<String, WindowInstance> getWindowInstanceMap() {
+ return windowInstanceMap;
+ }
+
+ public void setWindowInstanceMap(
+ ConcurrentHashMap<String, WindowInstance> windowInstanceMap) {
+ this.windowInstanceMap = windowInstanceMap;
+ }
+
+ public PiplineRecieverAfterCurrentNode getFireReceiver() {
+ return fireReceiver;
+ }
+
+ @Override
+ public void setFireReceiver(
+ PiplineRecieverAfterCurrentNode fireReceiver) {
+ this.fireReceiver = fireReceiver;
+ }
+
+ @Override
+ public boolean isSynchronous() {
+ return false;
+ }
+
+ public Map<String, String> getHavingMap() {
+ return havingMap;
+ }
+
+ public void setHavingMap(Map<String, String> havingMap) {
+ this.havingMap = havingMap;
+ }
+
+ public int getSlideInterval() {
+ return slideInterval;
+ }
+
+ public void setSlideInterval(int slideInterval) {
+ this.slideInterval = slideInterval;
+ }
+
+ public String getSizeVariable() {
+ return sizeVariable;
+ }
+
+ public Integer getSizeAdjust() {
+ return sizeAdjust;
+ }
+
+ public void setSizeAdjust(Integer sizeAdjust) {
+ this.sizeAdjust = sizeAdjust;
+ }
+
+ public Integer getSlideAdjust() {
+ return slideAdjust;
+ }
+
+ public void setSlideAdjust(Integer slideAdjust) {
+ this.slideAdjust = slideAdjust;
+ }
+
+ public String getSlideVariable() {
+ return slideVariable;
+ }
+
+ public void setSlideVariable(String slideVariable) {
+ this.slideVariable = slideVariable;
+ }
+
+ public String getReduceSerializeValue() {
+ return reduceSerializeValue;
+ }
+
+ public void setReduceSerializeValue(String reduceSerializeValue) {
+ this.reduceSerializeValue = reduceSerializeValue;
+ }
+
+ public IReducer getReducer() {
+ return reducer;
+ }
+
+ public void setReducer(IReducer reducer) {
+ this.reducer = reducer;
+ byte[] bytes = InstantiationUtil.serializeObject(reducer);
+ this.reduceSerializeValue=Base64Utils.encode(bytes);
+ }
+
+ public int getTimeUnitAdjust() {
+ return timeUnitAdjust;
+ }
+
+ public void setTimeUnitAdjust(int timeUnitAdjust) {
+ this.timeUnitAdjust = timeUnitAdjust;
+ }
+
+ public boolean isLocalStorageOnly() {
+ return isLocalStorageOnly;
+ }
+
+ public void setLocalStorageOnly(boolean localStorageOnly) {
+ isLocalStorageOnly = localStorageOnly;
+ }
+
+ public int getFireMode() {
+ return fireMode;
+ }
+
+ public void setFireMode(int fireMode) {
+ this.fireMode = fireMode;
+ }
+
+ public void removeInstanceFromMap(WindowInstance windowInstance) {
+ this.windowInstanceMap.remove(windowInstance.createWindowInstanceId());
+
+ }
+
+ @Override
+ public WindowCache getWindowCache() {
+ return windowCache;
+ }
+
+ public WindowStorage getStorage() {
+ return storage;
+ }
+
+ public WindowRireSource getWindowFireSource() {
+ return windowFireSource;
+ }
+
+ public IWindowMaxValueManager getWindowMaxValueManager() {
+ return windowMaxValueManager;
+ }
+
+ public Long getMsgMaxGapSecond() {
+ return msgMaxGapSecond;
+ }
+
+ public void setMsgMaxGapSecond(Long msgMaxGapSecond) {
+ this.msgMaxGapSecond = msgMaxGapSecond;
+ }
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java
new file mode 100644
index 0000000..1cc2aea
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java
@@ -0,0 +1,154 @@
+/*
+ * 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 org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV;
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.utils.DateUtil;
+import org.apache.rocketmq.streams.window.model.WindowInstance;
+import org.apache.rocketmq.streams.window.operator.AbstractWindow;
+
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * 只支持 时间去重的场景,日志是按系统时间顺序,所以不落盘。需要设置groupByFieldName和rowNumerName字段
+ */
+public class OverWindow extends AbstractWindow {
+
+ private static int MAX_SIZE = 1000000;
+ protected transient IntValueKV intValueKV;
+ protected transient WindowInstance windowInstance;
+ /**
+ * 需要把生成的序列号返回设置到message,这个是序列号对应的名字
+ */
+ protected String rowNumerName;
+
+ /**
+ * 针对这个窗口实例完成计算,实际上是写入了缓存,在flush时完成真正的计算。写入缓存时把上下文(header,windowinstance,window)保存在消息中
+ *
+ * @param message
+ * @param context
+ */
+ @Override
+ public AbstractContext<IMessage> doMessage(IMessage message, AbstractContext context) {
+ String key = generateShuffleKey(message);
+ createWindowInstanceByDate(new Date());
+ Integer value = intValueKV.get(key);
+ if (value == null) {
+ synchronized (this) {
+ value = intValueKV.get(key);
+ if (value == null) {
+ value = 1;
+ intValueKV.put(key, value);
+ }
+ }
+
+ }
+ message.getMessageBody().put(rowNumerName, value);
+ /**
+ * 如果超过最大值,直接归0
+ */
+ if (intValueKV.getSize() > MAX_SIZE) {
+ synchronized (this) {
+ if (intValueKV.getSize() > MAX_SIZE) {
+ intValueKV = new IntValueKV(MAX_SIZE);
+ }
+ }
+ }
+ return context;
+ }
+
+ /**
+ * 如果时间内无instance,创建,如果不在现有的instance中,现有的失效,重新创建
+ *
+ * @param date
+ * @return
+ */
+ protected void createWindowInstanceByDate(Date date) {
+ if (windowInstance == null) {
+ synchronized (this) {
+ if (windowInstance == null) {
+ windowInstance = createWindowInstance(date);
+ intValueKV = new IntValueKV(MAX_SIZE);
+ return;
+ }
+ }
+ }
+ String dateStr = DateUtil.format(date);
+ if (dateStr.compareTo(windowInstance.getStartTime()) >= 0 && dateStr.compareTo(windowInstance.getEndTime()) <= 0) {
+ return;
+ } else {
+ synchronized (this) {
+ windowInstance = createWindowInstance(date);
+ intValueKV = new IntValueKV(MAX_SIZE);
+ }
+ }
+ }
+
+ /**
+ * 根据日期创建window instance
+ *
+ * @param date
+ * @return
+ */
+ protected WindowInstance createWindowInstance(Date date) {
+ List<Date> instanceStartTimes = DateUtil.getWindowBeginTime(date.getTime(), slideInterval, sizeInterval);
+ Date instanceStartTime = instanceStartTimes.get(0);
+ WindowInstance windowInstance = new WindowInstance();
+ windowInstance.setStartTime(DateUtil.format(instanceStartTime));
+ Date endDate = DateUtil.addMinute(instanceStartTime, sizeInterval);
+ windowInstance.setEndTime(DateUtil.format(endDate));
+ return windowInstance;
+ }
+
+ @Override
+ public boolean isSynchronous() {
+ return true;
+ }
+
+ @Override
+ protected boolean initConfigurable() {
+ return true;
+ }
+
+ @Override
+ public Class getWindowBaseValueClass() {
+ return null;
+ }
+
+ @Override
+ public int fireWindowInstance(WindowInstance windowInstance, Map<String, String> queueId2Offsets) {
+ return 0;
+ }
+
+ @Override
+ public void clearFire(List<WindowInstance> windowInstances) {
+
+ }
+
+ public String getRowNumerName() {
+ return rowNumerName;
+ }
+
+ public void setRowNumerName(String rowNumerName) {
+ this.rowNumerName = rowNumerName;
+ }
+
+}
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
new file mode 100644
index 0000000..fcdae1a
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionWindow.java
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.window.operator.impl;
+
+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;
+
+import java.util.*;
+
+/**
+ * 实现思路: 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<String, String> queueId2Offset) {
+ List<WindowValue> fireWindowValues = new ArrayList<>();
+ int fireCount = 0;
+ //for(String queueId:queueIds){
+ WindowBaseValueIterator<WindowBaseValue> 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 = getRealFireTime(instance);
+ 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());
+ shuffleChannel.addNeedFlushWindowInstance(instance);
+ 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<String, WindowBaseValue> newWindowValues, Map<String, WindowBaseValue> exisitWindowValues, WindowInstance windowInstance, String queueId) {
+ List<String> oldKeys = new ArrayList<>();
+ Map<String, WindowBaseValue> partionNumOrders = new HashMap<>();//需要基于key前缀排序partitionnum
+ for (WindowBaseValue windowBaseValue : exisitWindowValues.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);
+ }
+
+ }
+ //
+ for (WindowBaseValue windowBaseValue : newWindowValues.values()) {
+ WindowValue windowValue = (WindowValue)windowBaseValue;
+ String oldPartitionNumKey = createStoreKey(getOrderBypPrefix() + queueId, MapKeyUtil.createKey(getOrderBypFieldName(windowValue), windowValue.getGroupBy()), 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(exisitWindowValues);
+ storage.multiPut(newWindowValues);
+ storage.multiPut(partionNumOrders);
+ }
+
+ @Override
+ public List<WindowInstance> 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<WindowInstance> windowInstances = new ArrayList<>();
+ windowInstances.add(windowInstance);
+ return windowInstances;
+ }
+
+ /**
+ * clear has fired window value
+ *
+ * @param windowValues
+ * @param queueId
+ * @param instance
+ */
+ protected void clearWindowValues(List<WindowValue> windowValues, String queueId, WindowInstance instance) {
+ if (windowValues == null || windowValues.size() == 0) {
+ return;
+ }
+ Set<String> deleteKeys = new HashSet<>();
+ List<String> 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);
+ }
+
+ /**
+ * 获取实际触发的时间,增加了固定延迟和water marker
+ *
+ * @param windowInstance
+ * @return
+ */
+ @Override
+ public Date getRealFireTime(WindowInstance windowInstance) {
+ Date fireDate = DateUtil.parse(windowInstance.getFireTime());
+ return fireDate;
+ }
+
+ /**
+ * 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.getWindowMaxValueManager().updateWindowEventTime(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
new file mode 100644
index 0000000..aaddfb0
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java
@@ -0,0 +1,451 @@
+/*
+ * 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 org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.context.IMessage;
+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.db.driver.batchloader.IRowOperator;
+import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
+import org.apache.rocketmq.streams.window.model.WindowInstance;
+import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow;
+import org.apache.rocketmq.streams.window.operator.AbstractWindow;
+import org.apache.rocketmq.streams.window.state.WindowBaseValue;
+import org.apache.rocketmq.streams.window.state.impl.WindowValue;
+import org.apache.rocketmq.streams.window.storage.IWindowStorage;
+import org.apache.rocketmq.streams.window.storage.ShufflePartitionManager;
+import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator;
+
+import java.util.*;
+import java.util.Map.Entry;
+
+public class WindowOperator extends AbstractShuffleWindow {
+
+ private static final String ORDER_BY_SPLIT_NUM="_order_by_split_num_";//key=_order;queueid,windowinstanceid,partitionNum
+
+ public WindowOperator() {
+ super();
+ }
+
+ @Deprecated
+ public WindowOperator(String timeFieldName, int windowPeriodMinute) {
+ super();
+ super.timeFieldName = timeFieldName;
+ super.sizeInterval = windowPeriodMinute;
+ }
+
+ @Deprecated
+ public WindowOperator(String timeFieldName, int windowPeriodMinute, String calFieldName) {
+ super();
+ super.timeFieldName = timeFieldName;
+ super.sizeInterval = windowPeriodMinute;
+ }
+
+ public WindowOperator(int sizeInterval, String groupByFieldName, Map<String, String> select) {
+ this.sizeInterval = sizeInterval;
+ this.slideInterval = sizeInterval;
+ this.groupByFieldName = groupByFieldName;
+ this.setSelectMap(select);
+ }
+
+ // protected transient AtomicInteger shuffleCount=new AtomicInteger(0);
+ //protected transient AtomicInteger fireCountAccumulator=new AtomicInteger(0);
+ @Override
+ public int fireWindowInstance(WindowInstance instance, String queueId,Map<String,String> queueId2Offset) {
+ List<WindowValue> windowValues=new ArrayList<>();
+ int fireCount=0;
+ //Set<String> currentQueueIds=new HashSet<>(queueIds);
+ long startTime=System.currentTimeMillis();
+ int sendCost=0;
+ int currentCount=0;
+ //for(String queueId:currentQueueIds){
+ WindowBaseValueIterator<WindowBaseValue> 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));
+ }
+ }
+ while (it.hasNext()){
+ WindowBaseValue windowBaseValue=it.next();
+ if(windowBaseValue==null){
+ continue;
+ }
+ //WindowValue windowValue=(WindowValue)windowBaseValue;
+ //Integer currentValue=(Integer)windowValue.getComputedColumnResultByKey("total");
+ //if(currentValue==null){
+ // currentValue=0;
+ //}
+ // fireCountAccumulator.addAndGet(currentValue);
+ // WindowValue windowValue=(WindowValue)windowBaseValue;
+ windowValues.add((WindowValue)windowBaseValue);
+ if(windowValues.size()>=windowCache.getBatchSize()){
+ long sendFireCost=System.currentTimeMillis();
+ sendFireMessage(windowValues,queueId);
+ sendCost+=(System.currentTimeMillis()-sendFireCost);
+ fireCount+=windowValues.size();
+ windowValues=new ArrayList<>();
+ }
+
+ }
+ if(windowValues.size()>0){
+ long sendFireCost=System.currentTimeMillis();
+ sendFireMessage(windowValues,queueId);
+ sendCost+=(System.currentTimeMillis()-sendFireCost);
+ fireCount+=windowValues.size();
+ }
+ // }
+
+ //if(fireCount==0){
+ // System.out.println(shuffleWindowInstanceId2MsgCount.get(instance.createWindowInstanceId()));
+ //}
+ // System.out.println("fire count is "+fireCountAccumulator.get());
+ //long clearStart=System.currentTimeMillis();
+ clearFire(instance);
+ // System.out.println("=============== fire cost is "+(System.currentTimeMillis()-startTime)+"send cost is "+sendCost+" clear cost is "+(System.currentTimeMillis()-clearStart));
+ return fireCount;
+ }
+
+ protected transient Map<String,Integer> shuffleWindowInstanceId2MsgCount=new HashMap<>();
+
+ @Override
+ public void shuffleCalculate(List<IMessage> messages, WindowInstance instance, String queueId) {
+ Map<String,List<IMessage>> groupBy=groupByGroupName(messages);
+ Set<String> groupByKeys=groupBy.keySet();
+ List<String> storeKeys=new ArrayList<>();
+ for(String groupByKey:groupByKeys){
+ String storeKey=createStoreKey(queueId,groupByKey,instance);
+ storeKeys.add(storeKey);
+ }
+ Map<String,WindowBaseValue> exisitWindowValues=new HashMap<>();
+ Map<String,WindowBaseValue> newWindowValues=new HashMap<>();
+
+ //从存储中,查找window value对象,value是对象的json格式
+ Map<String, WindowBaseValue> key2WindowValues=storage.multiGet(getWindowBaseValueClass(),storeKeys,instance.createWindowInstanceId(),queueId);
+ Iterator<Entry<String, List<IMessage>>> it = groupBy.entrySet().iterator();
+ while (it.hasNext()){
+ Entry<String, List<IMessage>> entry=it.next();
+ String groupByKey=entry.getKey();
+ String storeKey=createStoreKey(queueId,groupByKey,instance);
+ WindowValue windowValue=(WindowValue)key2WindowValues.get(storeKey);
+ List<IMessage> msgs=entry.getValue();
+ if(windowValue==null){
+ windowValue=createWindowValue(queueId,groupByKey,instance);
+ newWindowValues.put(storeKey,windowValue);
+ }else {
+ exisitWindowValues.put(storeKey,windowValue);
+ }
+
+ windowValue.incrementUpdateVersion();
+ //Integer origValue=(Integer)windowValue.getComputedColumnResultByKey("total");
+ //if(origValue==null){
+ // origValue=0;
+ //}
+ if(msgs!=null){
+ for(IMessage message:msgs){
+ windowValue.calculate(this,message);
+ }
+ }
+ //
+ //Integer currentValue=(Integer)windowValue.getComputedColumnResultByKey("total");
+ //if(currentValue==null){
+ // currentValue=0;
+ //}
+ //shuffleCount.addAndGet(-origValue);
+ //shuffleCount.addAndGet(currentValue);
+ //System.out.println("==========shuffle count is "+shuffleCount.get());
+
+ }
+
+ saveStorage(newWindowValues,exisitWindowValues,instance,queueId);
+ //Integer count=shuffleWindowInstanceId2MsgCount.get(instance.createWindowInstanceId());
+ //if(count==null){
+ // count=0;
+ //}
+ //count+=messages.size();
+ //shuffleWindowInstanceId2MsgCount.put(instance.createWindowInstanceId(),count);
+ }
+
+ protected void saveStorage(Map<String, WindowBaseValue> newWindowValues, Map<String, WindowBaseValue> exisitWindowValues,WindowInstance windowInstance,String queueId) {
+ String windowInstanceId=windowInstance.createWindowInstanceId();
+ Map<String, WindowBaseValue> allWindowValues=new HashMap<>();
+ allWindowValues.putAll(newWindowValues);
+ allWindowValues.putAll(exisitWindowValues);
+ storage.multiPut(allWindowValues,windowInstanceId,queueId);
+ Map<String,WindowBaseValue> partionNumOrders=new HashMap<>();//需要基于key前缀排序partitionnum
+ for(WindowBaseValue windowBaseValue:allWindowValues.values()){
+ WindowValue windowValue=(WindowValue)windowBaseValue;
+ String partitionNumKey=createStoreKey(getOrderBypPrefix()+queueId,MapKeyUtil.createKey(getOrderBypFieldName(windowValue),windowValue.getGroupBy()),windowInstance);
+ partionNumOrders.put(partitionNumKey,windowValue);
+ }
+ storage.getLocalStorage().multiPut(partionNumOrders);
+ }
+
+ @Override
+ public Class getWindowBaseValueClass() {
+ return WindowValue.class;
+ }
+
+ /**
+ * 按group name 进行分组
+ *
+ * @param messages
+ * @return
+ */
+ protected Map<String, List<IMessage>> groupByGroupName(List<IMessage> messages) {
+ if(messages==null||messages.size()==0){
+ return new HashMap<>();
+ }
+ Map<String,List<IMessage>> groupBy=new HashMap<>();
+ for(IMessage message:messages){
+ String groupByValue=generateShuffleKey(message);
+ if(StringUtil.isEmpty(groupByValue)){
+ groupByValue="<null>";
+ }
+ List<IMessage> messageList=groupBy.get(groupByValue);
+ if(messageList==null){
+ messageList=new ArrayList<>();
+ groupBy.put(groupByValue,messageList);
+ }
+ messageList.add(message);
+ }
+ return groupBy;
+ }
+
+ /**
+ * 创建新的window value对象
+ *
+ * @param groupBy
+ * @param instance
+ * @return
+ */
+ protected WindowValue createWindowValue(String queueId,String groupBy, WindowInstance instance) {
+ WindowValue windowValue = new WindowValue();
+ windowValue.setNameSpace(getNameSpace());
+ windowValue.setConfigureName(getConfigureName());
+ windowValue.setStartTime(instance.getStartTime());
+ windowValue.setEndTime(instance.getEndTime());
+ windowValue.setFireTime(instance.getFireTime());
+ windowValue.setGroupBy(groupBy==null?"":groupBy);
+ windowValue.setMsgKey(StringUtil.createMD5Str(MapKeyUtil.createKey(queueId, instance.createWindowInstanceId(), groupBy)));
+ String shuffleId=shuffleChannel.getChannelQueue(groupBy).getQueueId();
+ windowValue.setPartitionNum(createPartitionNum(windowValue,queueId,instance));
+ windowValue.setPartition(shuffleId);
+ windowValue.setWindowInstancePartitionId(instance.getWindowInstanceKey());
+ windowValue.setWindowInstanceId(instance.getWindowInstanceKey());
+ return windowValue;
+
+ }
+
+ protected long createPartitionNum(WindowValue windowValue, String shuffleId, WindowInstance instance) {
+ return incrementAndGetSplitNumber(instance,shuffleId);
+ }
+
+ /**
+ * 创建存储key
+ *
+ * @param groupByKey
+ * @param windowInstance
+ * @return
+ */
+ protected static String createStoreKey(String shuffleId, String groupByKey,WindowInstance windowInstance){
+ String storeKey = MapKeyUtil.createKey(shuffleId, windowInstance.createWindowInstanceId(), groupByKey);
+ return storeKey;
+ }
+
+ /**
+ * 需要排序的前缀
+ *
+ * @return
+ */
+ protected static String getOrderBypPrefix(){
+ return ORDER_BY_SPLIT_NUM;
+ }
+
+ /**
+ * 需要排序的字段值
+ *
+ * @return
+ */
+ protected static String getOrderBypFieldName(WindowValue windowValue){
+ return windowValue.getPartitionNum()+"";
+ }
+
+ /**
+ * 删除掉触发过的数据
+ *
+ * @param windowInstances
+ */
+ @Override
+ public void clearFire(List<WindowInstance> windowInstances) {
+ //clear window instances in db
+
+ //clear window value in db
+ for (WindowInstance windowInstance : windowInstances) {
+ Set<String> patitionNums=new HashSet<>();
+ patitionNums.add(getOrderBypPrefix()+ windowInstance.getSplitId());
+ Set<String> queueIds=new HashSet<>();
+ queueIds.add(windowInstance.getSplitId());
+ boolean canClear=false;
+ if(fireMode!=2){
+ canClear=true;
+ }
+ if(fireMode==2){
+ Date endTime=DateUtil.parse(windowInstance.getEndTime());
+ Date lastDate = DateUtil.addSecond(endTime, waterMarkMinute * timeUnitAdjust);
+
+ if((windowInstance.getLastMaxUpdateTime()>lastDate.getTime())){
+ canClear=true;
+ }
+ }
+
+ if(canClear){
+ this.windowInstanceMap.remove(windowInstance.createWindowInstanceId());
+ if(!isLocalStorageOnly){
+ WindowInstance.clearInstances(windowInstances);
+ }
+ shuffleChannel.clearCache(windowInstance);
+ ShufflePartitionManager.getInstance().clearWindowInstance(windowInstance.createWindowInstanceId());
+ storage.delete(windowInstance.createWindowInstanceId(),queueIds,getWindowBaseValueClass());
+ storage.getLocalStorage().delete(windowInstance.createWindowInstanceId(),patitionNums,getWindowBaseValueClass());
+ }
+
+ }
+
+ }
+
+ @Override
+ public void clearCache(String queueId) {
+ getStorage().clearCache(shuffleChannel.getChannelQueue(queueId),getWindowBaseValueClass());
+ getStorage().clearCache(getOrderByQueue(queueId,getOrderBypPrefix()),getWindowBaseValueClass());
+ ShufflePartitionManager.getInstance().clearSplit(queueId);
+ }
+
+ public ISplit getOrderByQueue(String key,String prefix){
+ int index=shuffleChannel.hash(key);
+ ISplit targetQueue = shuffleChannel.getQueueList().get(index);
+ return new ISplit() {
+ @Override
+ public String getQueueId() {
+ return prefix+targetQueue.getQueueId();
+ }
+
+ @Override
+ public String getPlusQueueId() {
+ return prefix+targetQueue.getPlusQueueId();
+ }
+
+ @Override
+ public Object getQueue() {
+ return targetQueue.getQueue();
+ }
+
+ @Override
+ public int compareTo(Object o) {
+ return targetQueue.compareTo(o);
+ }
+
+ @Override
+ public String toJson() {
+ return targetQueue.toJson();
+ }
+
+ @Override
+ public void toObject(String jsonString) {
+ targetQueue.toObject(jsonString);
+ }
+ };
+ }
+
+ public static void compareAndSet(WindowInstance windowInstance, IWindowStorage storage,List<WindowValue> windowValues){
+ if(windowValues==null||storage==null){
+ return;
+ }
+ synchronized (storage){
+ List<String> storeKeys = new ArrayList<>();
+ Map<String,WindowValue> windowValueMap=new HashMap<>();
+ for(WindowValue windowValue:windowValues){
+ String storeKey=createStoreKey(windowValue.getPartition(),windowValue.getGroupBy(),windowInstance);
+ storeKeys.add(storeKey);
+ windowValueMap.put(storeKey,windowValue);
+ String storeOrderKey=createStoreKey(windowValue.getPartition(),windowValue.getPartitionNum()+"",windowInstance);
+ windowValueMap.put(storeOrderKey,windowValue);
+ }
+ Map<String, WindowBaseValue> valueMap = storage.multiGet(WindowValue.class,storeKeys);
+ if(valueMap==null||valueMap.size()==0){
+ storage.multiPut(windowValueMap);
+ return;
+ }
+ Iterator<Entry<String, WindowBaseValue>> it = valueMap.entrySet().iterator();
+
+ while (it.hasNext()){
+ Entry<String, WindowBaseValue> entry=it.next();
+ String storeKey=entry.getKey();
+ WindowBaseValue localValue=entry.getValue();
+ WindowValue windowValue=windowValueMap.get(storeKey);
+ if (windowValue.getUpdateVersion() <= localValue.getUpdateVersion()) {
+ windowValueMap.remove(storeKey);
+ }
+ }
+ if(CollectionUtil.isNotEmpty(windowValueMap)){
+ storage.multiPut(windowValueMap);
+ }
+ }
+ }
+
+ public static class WindowRowOperator implements IRowOperator {
+
+ protected WindowInstance windowInstance;
+ protected String spiltId;
+ protected AbstractWindow window;
+
+ public WindowRowOperator(WindowInstance windowInstance,String spiltId,AbstractWindow window){
+ this.windowInstance=windowInstance;
+ this.spiltId=spiltId;
+ this.window=window;
+ }
+
+ @Override
+ public synchronized void doProcess(Map<String, Object> row) {
+ WindowValue windowValue = ORMUtil.convert(row, WindowValue.class);
+ List<String> keys = new ArrayList<>();
+ String storeKey=createStoreKey(spiltId,windowValue.getGroupBy(),windowInstance);
+ keys.add(storeKey);
+ String storeOrderKey=createStoreKey(getOrderBypPrefix()+windowValue.getPartition(),MapKeyUtil.createKey(getOrderBypFieldName(windowValue),windowValue.getGroupBy()),windowInstance);
+ Map<String, WindowBaseValue> valueMap = window.getStorage().getLocalStorage().multiGet(WindowValue.class,keys);
+ if (CollectionUtil.isEmpty(valueMap)) {
+ Map<String, WindowBaseValue> map = new HashMap<>(4);
+
+ map.put(storeKey, windowValue);
+ map.put(storeOrderKey, windowValue);
+ window.getStorage().getLocalStorage().multiPut(map);
+ return;
+ }
+ WindowValue localValue = (WindowValue)valueMap.values().iterator().next();
+ if (windowValue.getUpdateVersion() > localValue.getUpdateVersion()) {
+ Map<String, WindowBaseValue> map = new HashMap<>();
+ map.put(storeKey, windowValue);
+ map.put(storeOrderKey, windowValue);
+ window.getStorage().getLocalStorage().multiPut(map);
+ }
+ }
+ }
+
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/DBOperator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/DBOperator.java
new file mode 100644
index 0000000..b8bab5d
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/DBOperator.java
@@ -0,0 +1,427 @@
+/*
+ * 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.join;
+
+import com.alibaba.fastjson.JSONArray;
+import com.alibaba.fastjson.JSONObject;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.apache.rocketmq.streams.common.context.MessageHeader;
+import org.apache.rocketmq.streams.common.utils.DateUtil;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
+import org.apache.rocketmq.streams.window.state.impl.JoinLeftState;
+import org.apache.rocketmq.streams.window.state.impl.JoinRightState;
+import org.apache.rocketmq.streams.window.state.impl.JoinState;
+
+import java.security.MessageDigest;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+
+public class DBOperator implements Operator {
+
+ private static final Log LOG = LogFactory.getLog(DBOperator.class);
+
+ /**
+ * 根据join流对原始数据进行处理并入库
+ *
+ * @param messageList
+ * @param joinType
+ */
+ public void addMessage(List<IMessage> messageList, String joinType) {
+ List<JoinLeftState> joinLeftStates = new ArrayList<JoinLeftState>();
+ List<JoinRightState> joinRightStates = new ArrayList<JoinRightState>();
+ for (IMessage message : messageList) {
+ String routeLabel = message.getHeader().getMsgRouteFromLable();
+ JoinState state = dealMessge(message, routeLabel);
+ if ("left".equalsIgnoreCase(routeLabel)) {
+ joinLeftStates.add((JoinLeftState)state);
+ } else if ("right".equalsIgnoreCase(routeLabel)) {
+ joinRightStates.add((JoinRightState)state);
+ }
+ // joinStates.add(state);
+ }
+
+ if (joinLeftStates.size() > 0) {
+ ORMUtil.batchReplaceInto(joinLeftStates);
+ }
+ if (joinRightStates.size() > 0) {
+ ORMUtil.batchReplaceInto(joinRightStates);
+ }
+
+ }
+
+ /**
+ * 生成joinstate对象
+ *
+ * @param message
+ * @param routeLabel
+ * @return
+ */
+ public JoinState dealMessge(IMessage message, String routeLabel) {
+
+ JSONObject window = message.getMessageBody().getJSONObject("Window");
+ String windowName = window.getString("configureName");
+ String windowNameSpace = window.getString("nameSpace");
+ MessageHeader header = message.getHeader();
+ String queueId = "_Dipper";
+ String offset = System.nanoTime() + "";
+ if (header.getQueueId() != null) {
+ queueId = header.getQueueId();
+ }
+ if (header.isEmptyOffset()) {
+ header.setOffset(offset);
+ offset = header.getOffset();
+ }
+
+ String messageId = windowNameSpace + "_" + windowName + "_" + queueId + "_" + offset;
+
+ List<String> leftJoinFieldNames = window.getJSONArray("leftJoinFieldNames") != null ?
+ window.getJSONArray("leftJoinFieldNames").toJavaList(String.class) :
+ new ArrayList<>();
+ List<String> rightJoinFieldNames = window.getJSONArray("rightJoinFieldNames") != null ?
+ window.getJSONArray("rightJoinFieldNames").toJavaList(String.class) :
+ new ArrayList<>();
+
+ String messageKey = generateKey(message.getMessageBody(), routeLabel, leftJoinFieldNames, rightJoinFieldNames);
+ JSONObject messageBody = (JSONObject)message.getMessageBody().clone();
+ messageBody.remove("WindowInstance");
+ messageBody.remove("Window");
+
+ JoinState state = null;
+ if ("left".equalsIgnoreCase(routeLabel)) {
+ state = new JoinLeftState();
+ } else if ("right".equalsIgnoreCase(routeLabel)) {
+ state = new JoinRightState();
+ }
+
+ state.setGmtCreate(new Date());
+ state.setGmtModified(new Date());
+ state.setWindowName(windowName);
+ state.setWindowNameSpace(windowNameSpace);
+ state.setMessageId(messageId);
+ state.setMessageKey(messageKey);
+ state.setMessageTime(new Date());
+ state.setMessageBody(messageBody.toJSONString());
+
+ return state;
+ }
+
+ /**
+ * 根据join条件生成消息比对key值
+ *
+ * @param messageBody
+ * @param joinLabel
+ * @param leftJoinFieldNames
+ * @param rightJoinFieldNames
+ * @return
+ */
+ public static String generateKey(JSONObject messageBody, String joinLabel, List<String> leftJoinFieldNames, List<String> rightJoinFieldNames) {
+ StringBuffer buffer = new StringBuffer();
+ if ("left".equalsIgnoreCase(joinLabel)) {
+ for (String field : leftJoinFieldNames) {
+ String value = messageBody.getString(field);
+ buffer.append(value).append("_");
+ }
+ } else {
+ for (String field : rightJoinFieldNames) {
+ String[] rightFields = field.split("\\.");
+ if (rightFields.length > 1) {
+ field = rightFields[1];
+ }
+ String value = messageBody.getString(field);
+ buffer.append(value).append("_");
+ }
+ }
+
+ return MD5(buffer.toString());
+ }
+
+ public static String MD5(String s) {
+ char hexDigits[] = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'};
+
+ try {
+ byte[] btInput = s.getBytes();
+ // 获得MD5摘要算法的 MessageDigest 对象
+ MessageDigest mdInst = MessageDigest.getInstance("MD5");
+ // 使用指定的字节更新摘要
+ mdInst.update(btInput);
+ // 获得密文
+ byte[] md = mdInst.digest();
+ // 把密文转换成十六进制的字符串形式
+ int j = md.length;
+ char str[] = new char[j * 2];
+ int k = 0;
+ for (int i = 0; i < j; i++) {
+ byte byte0 = md[i];
+ str[k++] = hexDigits[byte0 >>> 4 & 0xf];
+ str[k++] = hexDigits[byte0 & 0xf];
+ }
+ return new String(str);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new RuntimeException(e.getMessage(), e);
+ }
+ }
+
+ public List<Map<String, Object>> dealJoin(IMessage message) {
+ List<Map<String, Object>> result = new ArrayList<>();
+ JSONObject messageBody = message.getMessageBody();
+ JSONObject msg = messageBody.getJSONObject("msg");
+ String routeLabel = messageBody.getString("routeLabel");
+ JSONArray windowInstances = msg.getJSONArray("WindowInstance");
+ JSONObject windowInstance = null;
+ if (windowInstances != null && windowInstances.size() > 0) {
+ windowInstance = windowInstances.getJSONObject(0);
+ } else {
+ return result;
+ }
+
+ Integer retainWindowCount = messageBody.getInteger("retainWindowCount");
+ Integer sizeInterval = messageBody.getInteger("sizeInterval");
+
+ List<JSONObject> tmp = new ArrayList<>();
+ if ("left".equalsIgnoreCase(routeLabel)) {
+ String endTime = windowInstance.getString("endTime");
+ String startTime = addTime(windowInstance.getString("startTime"), TimeUnit.MINUTES, -retainWindowCount * sizeInterval);
+ String tableName = "join_right_state";
+ String messageKey = messageBody.getString("messageKey");
+ String windowName = windowInstance.getString("windowName");
+ String windowNameSpace = windowInstance.getString("windowNameSpace");
+ tmp = getJoinData(tableName, messageKey, windowName, windowNameSpace, startTime, endTime);
+
+ } else if ("right".equalsIgnoreCase(routeLabel)) {
+ // String startTime = addTime(windowInstance.getString("startTime"), TimeUnit.MINUTES, -sizeInterval);
+ String startTime = addTime(windowInstance.getString("startTime"), TimeUnit.MINUTES, -retainWindowCount * sizeInterval);
+ String endTime = addTime(windowInstance.getString("endTime"), TimeUnit.MINUTES, -sizeInterval);
+ String tableName = "join_left_state";
+ String messageKey = messageBody.getString("messageKey");
+ String windowName = windowInstance.getString("windowName");
+ String windowNameSpace = windowInstance.getString("windowNameSpace");
+ tmp = getJoinData(tableName, messageKey, windowName, windowNameSpace, startTime, endTime);
+ }
+
+ result = converToMapFromJson(tmp);
+ return result;
+
+ }
+
+ public List<JSONObject> connectJoin(IMessage message, List<Map<String, Object>> rows, String joinType, String rightAsName) {
+ List<JSONObject> result = new ArrayList<>();
+ if (rows.size() <= 0) {
+ return result;
+ }
+ if ("inner".equalsIgnoreCase(joinType)) {
+ result = connectInnerJoin(message, rows, rightAsName);
+ } else if ("left".equalsIgnoreCase(joinType)) {
+ result = connectLeftJoin(message, rows, rightAsName);
+ }
+ return result;
+ }
+
+ /**
+ * inner join 将匹配messageKey的各行与message进行连接
+ *
+ * @param message
+ * @param rows
+ * @return
+ */
+ public List<JSONObject> connectInnerJoin(IMessage message, List<Map<String, Object>> rows, String rightAsName) {
+ List<JSONObject> result = new ArrayList<>();
+ String routeLabel = message.getMessageBody().getString("routeLabel");
+ if ("left".equalsIgnoreCase(routeLabel)) {
+ JSONObject messageBody = message.getMessageBody().getJSONObject("msg");
+ for (Map<String, Object> raw : rows) {
+ // addAsName(raw, rightAsName);
+ JSONObject object = (JSONObject)messageBody.clone();
+ object.fluentPutAll(addAsName(raw, rightAsName));
+ result.add(object);
+ }
+ } else {
+ JSONObject messageBody = message.getMessageBody().getJSONObject("msg");
+ messageBody = (JSONObject)addAsName(messageBody, rightAsName);
+ for (Map<String, Object> raw : rows) {
+ JSONObject object = (JSONObject)messageBody.clone();
+ object.fluentPutAll(raw);
+ result.add(object);
+ }
+ }
+
+ return result;
+ }
+
+ private Map<String, Object> addAsName(Map<String, Object> raw, String rightAsName) {
+ Map<String, Object> asName = new HashMap<>();
+ for (Map.Entry<String, Object> tmp : raw.entrySet()) {
+ asName.put(rightAsName + "." + tmp.getKey(), tmp.getValue());
+ // raw.remove(tmp.getKey());
+ }
+ return asName;
+ }
+
+ public List<JSONObject> connectLeftJoin(IMessage message, List<Map<String, Object>> rows, String rightAsName) {
+ List<JSONObject> result = new ArrayList<>();
+ String routeLabel = message.getMessageBody().getString("routeLabel");
+ JSONObject messageBody = message.getMessageBody().getJSONObject("msg");
+ if ("left".equalsIgnoreCase(routeLabel)) {
+ if (rows != null && rows.size() > 0) {
+ for (Map<String, Object> raw : rows) {
+ // raw = addAsName(raw, rightAsName);
+ JSONObject object = (JSONObject)messageBody.clone();
+ object.fluentPutAll(addAsName(raw, rightAsName));
+ result.add(object);
+ }
+ return result;
+ } else {
+ result.add(messageBody);
+ }
+
+ } else {
+ if (rows != null && rows.size() > 0) {
+ messageBody = (JSONObject)addAsName(messageBody, rightAsName);
+ for (Map<String, Object> raw : rows) {
+ JSONObject object = (JSONObject)messageBody.clone();
+ object.fluentPutAll(raw);
+ result.add(object);
+ }
+ return result;
+ }
+ }
+
+ return result;
+
+ }
+
+ private List<Map<String, Object>> converToMapFromJson(List<JSONObject> list) {
+ List<Map<String, Object>> mapList = new ArrayList<>();
+ if (list != null && list.size() > 0) {
+ for (JSONObject object : list) {
+ Map<String, Object> tmp = object;
+ mapList.add(tmp);
+ }
+ }
+ return mapList;
+ }
+
+ public List<JSONObject> getJoinData(String tableName, String messageKey, String windowName, String windowNameSpace, String startTime, String endTime) {
+ Map<String, Object> paras = new HashMap<>();
+ paras.put("messageKey", messageKey);
+ paras.put("startTime", startTime);
+ paras.put("endTime", endTime);
+ paras.put("windowName", windowName);
+ paras.put("windowNameSpace", windowNameSpace);
+ // paras.put("tableName", tableName);
+ List<JoinState> result = new ArrayList<>();
+ if ("join_right_state".equalsIgnoreCase(tableName)) {
+ result = ORMUtil.queryForList("select message_body from join_right_state where message_key = #{messageKey} and window_name = #{windowName}" +
+ "and window_name_space = #{windowNameSpace} and gmt_create >= #{startTime} and gmt_create < #{endTime}", paras, JoinState.class);
+ } else if ("join_left_state".equalsIgnoreCase(tableName)) {
+ result = ORMUtil.queryForList("select message_body from join_left_state where message_key = #{messageKey} and window_name = #{windowName} " +
+ "and window_name_space = #{windowNameSpace} and gmt_create >= #{startTime} and gmt_create < #{endTime}", paras, JoinState.class);
+ }
+
+ List<JSONObject> bodys = new ArrayList<>();
+ for (JoinState tmp : result) {
+ try {
+ bodys.add(Message.parseObject(tmp.getMessageBody()));
+ } catch (Exception e) {
+ LOG.error("json parase error:", e);
+ }
+
+ }
+ return bodys;
+ }
+
+ public String addTime(String time, TimeUnit unit, int value) {
+ Date date = DateUtil.parseTime(time);
+ date = DateUtil.addDate(unit, date, value);
+ return DateUtil.format(date);
+ }
+
+ /**
+ * 根据window去除过期消息数据,消息去除时间为starttime加上窗口
+ *
+ * @param windowNameSpace
+ * @param windowName
+ * @param retainWindowCount
+ * @param sizeInterval
+ * @param startTime
+ */
+ public void cleanMessage(String windowNameSpace, String windowName, int retainWindowCount, int sizeInterval, String startTime) {
+ Map<String, Object> params = new HashMap<>();
+ String start = addTime(startTime, TimeUnit.MINUTES, -retainWindowCount * sizeInterval);
+ params.put("startTime", start);
+ params.put("windowNameSpace", windowNameSpace);
+ params.put("windowName", windowName);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("dboperata delete param is " + JSONObject.toJSONString(params));
+ }
+
+ List<JoinLeftState> joinLeftStates = ORMUtil.queryForList("select id from join_left_state where window_name_space = #{windowNameSpace} and " +
+ "window_name = #{windowName} and gmt_create < #{startTime}", params, JoinLeftState.class);
+ if (joinLeftStates != null && joinLeftStates.size() > 0) {
+ List<String> deleteIds = this.getDeleteIds(joinLeftStates);
+ for (String ids : deleteIds) {
+ // params.put("ids", ids);
+ ORMUtil.executeSQL("delete from join_left_state where id in (" + ids + ")", null);
+ }
+
+ }
+
+ List<JoinRightState> joinRightStates = ORMUtil.queryForList("select id from join_right_state where window_name_space = #{windowNameSpace} and " +
+ "window_name = #{windowName} and gmt_create < #{startTime}", params, JoinRightState.class);
+ if (joinRightStates != null && joinRightStates.size() > 0) {
+ List<String> deleteIds = this.getDeleteIds(joinRightStates);
+ for (String ids : deleteIds) {
+ // params.put("ids", ids);
+ ORMUtil.executeSQL("delete from join_right_state where id in (" + ids + ")", null);
+ }
+
+ }
+
+ }
+
+ private List<String> getDeleteIds(List<? extends JoinState> instances) {
+ List<String> deteleIds = new ArrayList<>();
+ if (instances == null || instances.size() == 0) {
+ return deteleIds;
+ }
+ int count = 1;
+
+ StringBuilder builder = new StringBuilder();
+ for (; count <= instances.size(); count++) {
+ builder.append(instances.get(count - 1).getId());
+ if (count % 1000 == 0) {
+ deteleIds.add(builder.toString());
+ builder = new StringBuilder();
+ } else {
+ if (count == instances.size()) {
+ deteleIds.add(builder.toString());
+ } else {
+ builder.append(",");
+ }
+
+ }
+ // count++;
+ }
+
+ return deteleIds;
+ }
+
+}
\ No newline at end of file
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java
new file mode 100644
index 0000000..4124ec6
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java
@@ -0,0 +1,553 @@
+/*
+ * 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.join;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow;
+import org.apache.rocketmq.streams.window.state.impl.JoinLeftState;
+import org.apache.rocketmq.streams.window.state.impl.JoinRightState;
+import org.apache.rocketmq.streams.window.state.impl.JoinState;
+import org.apache.rocketmq.streams.window.model.WindowInstance;
+import org.apache.rocketmq.streams.window.model.WindowCache;
+import org.apache.rocketmq.streams.window.state.WindowBaseValue;
+import org.apache.rocketmq.streams.dim.model.AbstractDim;
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.Context;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.apache.rocketmq.streams.common.context.MessageHeader;
+import org.apache.rocketmq.streams.common.utils.DateUtil;
+import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+
+public class JoinWindow extends AbstractShuffleWindow {
+
+ public static final String JOIN_KEY = "JOIN_KEY";
+
+ public static final String LABEL_LEFT = "left";
+
+ public static final String LABEL_RIGHT = "right";
+
+ //保存多少个周期的数据。比如window的滚动周期是5分钟,join需要1个小时数据,则retainWindowCount=12
+ protected int retainWindowCount = 4;
+ protected List<String> leftJoinFieldNames;//join等值条件中,左流的字段列表
+ protected List<String> rightJoinFieldNames;//join等值条件中,右流的字段列表
+ protected String rightAsName;//主要用于sql场景,默认右流都需要有别名。开发模式不需要
+
+ protected String joinType;//join类型,值为INNER,LEFT
+ protected String expression;//条件表达式。在存在非等值比较时使用
+ protected transient DBOperator joinOperator = new DBOperator();
+
+ @Override
+ protected boolean initConfigurable() {
+ // return super.initConfigurable();
+ super.initConfigurable();
+ doProcessAfterRefreshConfigurable(null);
+ return true;
+ }
+
+ // @Override
+ // protected void addPropertyToMessage(IMessage oriMessage, JSONObject oriJson){
+ // oriJson.put("AbstractWindow", this);
+ //
+ // }
+
+ @Override
+ protected int fireWindowInstance(WindowInstance instance, String shuffleId, Map<String, String> queueId2Offsets) {
+ List<WindowInstance> instances = new ArrayList<>();
+ instances.add(instance);
+ clearFire(instances);
+ return 0;
+ }
+
+ @Override
+ public void clearCache(String queueId) {
+
+ }
+
+ @Override
+ public void shuffleCalculate(List<IMessage> messages, WindowInstance instance, String queueId) {
+ Map<String, WindowBaseValue> joinLeftStates = new HashMap<>();
+ Map<String, WindowBaseValue> joinRightStates = new HashMap<>();
+ for (IMessage msg : messages) {
+ MessageHeader header = JSONObject.parseObject(msg.getMessageBody().
+ getString(WindowCache.ORIGIN_MESSAGE_HEADER), MessageHeader.class);
+ msg.setHeader(header);
+ String routeLabel = header.getMsgRouteFromLable();
+ String storeKey = createStoreKey(msg, routeLabel, instance);
+ JoinState state = createJoinState(msg, instance, routeLabel);
+ if ("left".equalsIgnoreCase(routeLabel)) {
+ joinLeftStates.put(storeKey, state);
+ } else if ("right".equalsIgnoreCase(routeLabel)) {
+ joinRightStates.put(storeKey, state);
+ }
+ }
+
+ if (joinLeftStates.size() > 0) {
+ storage.multiPut(joinLeftStates);
+ }
+ if (joinRightStates.size() > 0) {
+ storage.multiPut(joinRightStates);
+ }
+
+ for (IMessage msg : messages) {
+ MessageHeader header = JSONObject.parseObject(msg.getMessageBody().
+ getString(WindowCache.ORIGIN_MESSAGE_HEADER), MessageHeader.class);
+ String routeLabel = header.getMsgRouteFromLable();
+ // Map<String,WindowBaseValue> joinMessages = new HashMap<>();
+ String storeKeyPrefix = "";
+ Iterator<WindowBaseValue> iterator = null;
+ if (LABEL_LEFT.equalsIgnoreCase(routeLabel)) {
+ storeKeyPrefix = createStoreKeyPrefix(msg, LABEL_RIGHT, instance);
+ iterator = getMessageIterator(queueId, instance, msg, storeKeyPrefix, JoinRightState.class);
+ } else if (LABEL_RIGHT.equalsIgnoreCase(routeLabel)) {
+ storeKeyPrefix = createStoreKeyPrefix(msg, LABEL_LEFT, instance);
+ iterator = getMessageIterator(queueId, instance, msg, storeKeyPrefix, JoinLeftState.class);
+ }
+
+ // Iterator<WindowBaseValue> iterator = getMessageIterator(queueId, instance, msg, storeKeyPrefix, JoinState.class);
+ List<WindowBaseValue> tmpMessages = new ArrayList<>();
+ int count = 0;
+ while (iterator.hasNext()) {
+ tmpMessages.add(iterator.next());
+ count++;
+ if (count == 100) {
+ sendMessage(msg, tmpMessages);
+ tmpMessages.clear();
+ count = 0;
+ }
+ }
+ sendMessage(msg, tmpMessages);
+
+ }
+ }
+
+ private Iterator<WindowBaseValue> getMessageIterator(String queueId, WindowInstance instance,
+ IMessage msg, String keyPrefix, Class<? extends WindowBaseValue> clazz) {
+
+ List<WindowInstance> instances = new ArrayList<>();
+ for (Map.Entry<String, WindowInstance> entry : this.windowInstanceMap.entrySet()) {
+ instances.add(entry.getValue());
+ }
+ Iterator<WindowInstance> windowInstanceIter = instances.iterator();
+ return new Iterator<WindowBaseValue>() {
+ protected volatile boolean hasNext = true;
+ protected AtomicBoolean hasInit = new AtomicBoolean(false);
+ protected Iterator<WindowBaseValue> iterator = null;
+
+ @Override
+ public boolean hasNext() {
+ if (iterator != null && iterator.hasNext()) {
+ return true;
+ }
+ while (windowInstanceIter.hasNext()) {
+ WindowInstance instance = windowInstanceIter.next();
+ iterator = storage.loadWindowInstanceSplitData(null, queueId,
+ instance.createWindowInstanceId(),
+ keyPrefix,
+ clazz);
+ if (iterator != null && iterator.hasNext()) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public WindowBaseValue next() {
+ return iterator.next();
+ }
+ };
+
+ }
+
+ private Iterator<WindowBaseValue> getIterator(String queueId, String keyPrefix, WindowInstance instance, Class<? extends WindowBaseValue> clazz) {
+
+ List<WindowInstance> instances = new ArrayList<>();
+ for (Map.Entry<String, WindowInstance> entry : this.windowInstanceMap.entrySet()) {
+ instances.add(entry.getValue());
+ }
+ Iterator<WindowInstance> windowInstanceIter = instances.iterator();
+ return new Iterator<WindowBaseValue>() {
+ protected volatile boolean hasNext = true;
+ protected AtomicBoolean hasInit = new AtomicBoolean(false);
+ protected Iterator<WindowBaseValue> iterator = null;
+
+ @Override
+ public boolean hasNext() {
+ if (iterator != null && iterator.hasNext()) {
+ return true;
+ }
+ if (windowInstanceIter.hasNext()) {
+ WindowInstance instance = windowInstanceIter.next();
+ iterator = storage.loadWindowInstanceSplitData(null, queueId, instance.createWindowInstanceId(), keyPrefix, clazz);
+ if (iterator != null && iterator.hasNext()) {
+ return true;
+ } else {
+ return false;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public WindowBaseValue next() {
+ return iterator.next();
+ }
+ };
+
+ }
+
+ public List<JSONObject> connectJoin(IMessage message, List<Map<String, Object>> rows, String joinType, String rightAsName) {
+ List<JSONObject> result = new ArrayList<>();
+ if (rows.size() <= 0) {
+ return result;
+ }
+ if ("inner".equalsIgnoreCase(joinType)) {
+ result = connectInnerJoin(message, rows, rightAsName);
+ }
+ // else if ("left".equalsIgnoreCase(joinType)) {
+ // result = connectLeftJoin(message, rows, rightAsName);
+ // }
+ return result;
+ }
+
+ /**
+ * inner join 将匹配messageKey的各行与message进行连接
+ *
+ * @param message
+ * @param rows
+ * @return
+ */
+ public List<JSONObject> connectInnerJoin(IMessage message, List<Map<String, Object>> rows, String rightAsName) {
+ List<JSONObject> result = new ArrayList<>();
+ String routeLabel = message.getHeader().getMsgRouteFromLable();
+ if (LABEL_LEFT.equalsIgnoreCase(routeLabel)) {
+ JSONObject messageBody = message.getMessageBody();
+ for (Map<String, Object> raw : rows) {
+ // addAsName(raw, rightAsName);
+ JSONObject object = (JSONObject)messageBody.clone();
+ object.fluentPutAll(addAsName(raw, rightAsName));
+ result.add(object);
+ }
+ } else {
+ JSONObject messageBody = message.getMessageBody();
+ messageBody = addAsName(messageBody, rightAsName);
+ for (Map<String, Object> raw : rows) {
+ JSONObject object = (JSONObject)messageBody.clone();
+ object.fluentPutAll(raw);
+ result.add(object);
+ }
+ }
+
+ return result;
+ }
+
+ private JSONObject addAsName(Map<String, Object> raw, String rightAsName) {
+ JSONObject object = new JSONObject();
+ if (StringUtil.isEmpty(rightAsName)) {
+ return object.fluentPutAll(raw);
+ }
+ for (Map.Entry<String, Object> tmp : raw.entrySet()) {
+ object.put(rightAsName + "." + tmp.getKey(), tmp.getValue());
+ // raw.remove(tmp.getKey());
+ }
+ return object;
+ }
+
+ /**
+ * 生成join消息key值 全局唯一 key值构成结构为 shuffleId:shuffe split id, windowNamespace:窗口命名空间, windowName: 窗口名称, startTime:窗口开始时间, endTime:窗口结束时间, shuffleKey: join key值, routeLabel:消息左右流标记, orginQueueId:原始消息queueid, originOffset:原始消息offset
+ *
+ * @param message
+ * @param routeLabel
+ * @param windowInstance
+ * @return
+ */
+ protected String createStoreKey(IMessage message, String routeLabel, WindowInstance windowInstance) {
+ String shuffleKey = message.getMessageBody().getString(WindowCache.SHUFFLE_KEY);
+ String shuffleId = shuffleChannel.getChannelQueue(shuffleKey).getQueueId();
+ String orginQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID);
+ String originOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET);
+ String windowNamespace = getNameSpace();
+ String windowName = getConfigureName();
+ String startTime = windowInstance.getStartTime();
+ String endTime = windowInstance.getEndTime();
+ String storeKey = MapKeyUtil.createKey(shuffleId, windowNamespace, windowName, startTime, endTime, shuffleKey, routeLabel, orginQueueId, originOffset);
+ return storeKey;
+ }
+
+ protected String createStoreKeyPrefix(IMessage message, String routeLabel, WindowInstance windowInstance) {
+ String shuffleKey = message.getMessageBody().getString(WindowCache.SHUFFLE_KEY);
+ String storeKey = MapKeyUtil.createKey(shuffleKey, routeLabel);
+ return storeKey;
+ }
+
+ /**
+ * 根据左右流标志对原始消息进行封装
+ *
+ * @param message 原始消息
+ * @param instance
+ * @param routeLabel 左右流标志
+ * @return
+ */
+ private JoinState createJoinState(IMessage message, WindowInstance instance, String routeLabel) {
+ MessageHeader header = message.getHeader();
+ String queueId = "_Dipper";
+ String offset = System.nanoTime() + "";
+ if (header.getQueueId() != null) {
+ queueId = header.getQueueId();
+ }
+ if (header.isEmptyOffset()) {
+ header.setOffset(offset);
+ offset = header.getOffset();
+ }
+
+ String messageId = this.getNameSpace() + "_" + this.getConfigureName() + "_" + queueId + "_" + offset;
+
+ String messageKey = generateKey(message.getMessageBody(), routeLabel, this.leftJoinFieldNames, this.rightJoinFieldNames);
+ JSONObject messageBody = (JSONObject)message.getMessageBody().clone();
+ messageBody.remove("WindowInstance");
+ messageBody.remove("AbstractWindow");
+
+ JoinState state = null;
+ if ("left".equalsIgnoreCase(routeLabel)) {
+ state = new JoinLeftState();
+ } else if ("right".equalsIgnoreCase(routeLabel)) {
+ state = new JoinRightState();
+ }
+
+ state.setGmtCreate(new Date());
+ state.setGmtModified(new Date());
+ state.setWindowName(this.getConfigureName());
+ state.setWindowNameSpace(this.getNameSpace());
+ state.setMessageId(messageId);
+ state.setMessageKey(messageKey);
+ state.setMessageTime(new Date());
+ state.setMessageBody(messageBody.toJSONString());
+ state.setMsgKey(createStoreKey(message, routeLabel, instance));
+ String shuffleKey = message.getMessageBody().getString(WindowCache.SHUFFLE_KEY);
+ String shuffleId = shuffleChannel.getChannelQueue(shuffleKey).getQueueId();
+ state.setPartition(shuffleId);
+ state.setWindowInstanceId(instance.getWindowInstanceKey());
+ state.setPartitionNum(incrementAndGetSplitNumber(instance, shuffleId));
+ state.setWindowInstancePartitionId(instance.getWindowInstanceKey());
+
+ return state;
+ }
+
+ /**
+ * 根据join条件生成消息比对key值
+ *
+ * @param messageBody
+ * @param joinLabel
+ * @param leftJoinFieldNames
+ * @param rightJoinFieldNames
+ * @return
+ */
+ public static String generateKey(JSONObject messageBody, String joinLabel, List<String> leftJoinFieldNames, List<String> rightJoinFieldNames) {
+ StringBuffer buffer = new StringBuffer();
+ if ("left".equalsIgnoreCase(joinLabel)) {
+ for (String field : leftJoinFieldNames) {
+ String value = messageBody.getString(field);
+ buffer.append(value).append("_");
+ }
+ } else {
+ for (String field : rightJoinFieldNames) {
+ String[] rightFields = field.split("\\.");
+ if (rightFields.length > 1) {
+ field = rightFields[1];
+ }
+ String value = messageBody.getString(field);
+ buffer.append(value).append("_");
+ }
+ }
+
+ return StringUtil.createMD5Str(buffer.toString());
+ }
+
+ @Override
+ protected String generateShuffleKey(IMessage message) {
+ String routeLabel = message.getHeader().getMsgRouteFromLable();
+ String messageKey = generateKey(message.getMessageBody(), routeLabel, leftJoinFieldNames, rightJoinFieldNames);
+ return messageKey;
+ }
+
+ @Override
+ public Class getWindowBaseValueClass() {
+ return JoinState.class;
+ }
+
+ // @Override
+ // public void finishWindowProcessAndSend2Receiver(List<IMessage> messageList,WindowInstance windowInstance) {
+ // for (IMessage message : messageList) {
+ // List<Map<String, Object>> result = joinOperator.dealJoin(message);
+ // List<Map<String,Object>> rows = matchRows(message.getMessageBody(), result);
+ // String rightAsName = message.getMessageBody().getString("rightAsName");
+ // String joinType = message.getMessageBody().getString("joinType");
+ // List<JSONObject> connectMsgs = joinOperator.connectJoin(message, rows, joinType, rightAsName);
+ // for (int i=0; i < connectMsgs.size(); i++) {
+ // if (i == connectMsgs.size() -1) {
+ // sendMessage(connectMsgs.get(i), true);
+ // } else {
+ // sendMessage(connectMsgs.get(i), false);
+ // }
+ //
+ // }
+ //
+ // }
+ // //todo 完成处理
+ // //todo 发送消息到下一个节点 sendFireMessage();
+ // }
+
+ /**
+ * window触发后的清理工作
+ * @param windowInstances
+ */
+ /**
+ * 删除掉触发过的数据
+ *
+ * @param windowInstances
+ */
+ @Override
+ public void clearFire(List<WindowInstance> windowInstances) {
+
+ if (windowInstances != null && windowInstances.size() > 0) {
+ for (WindowInstance instance : windowInstances) {
+ // System.out.println("clear window is " + instance.getStartTime() + " " + instance.getEndTime() + " " + instance.windowName);
+
+ WindowInstance.cleanWindow(instance);
+ joinOperator.cleanMessage(instance.getWindowNameSpace(), instance.getWindowName(), this.getRetainWindowCount(),
+ this.getSizeInterval(), instance.getStartTime());
+ }
+ }
+ //todo windowinstace
+ //todo left+right
+ }
+
+ protected List<Map<String, Object>> matchRows(JSONObject msg, List<Map<String, Object>> rows) {
+
+ return AbstractDim.matchExpressionByLoop(rows.iterator(), expression, msg, true);
+ }
+
+ private List<Map<String, Object>> converToMapFromList(List<WindowBaseValue> rows) {
+ List<Map<String, Object>> joinMessages = new ArrayList<>();
+ for (WindowBaseValue value : rows) {
+ JSONObject obj = Message.parseObject(((JoinState)value).getMessageBody());
+ joinMessages.add((Map<String, Object>)obj);
+ }
+ return joinMessages;
+ }
+
+ /**
+ * 把触发的数据,发送到下一个节点
+ *
+ * @param message
+ * @param needFlush
+ */
+ protected void sendMessage(JSONObject message, boolean needFlush) {
+ Message nextMessage = new Message(message);
+ if (needFlush) {
+ nextMessage.getHeader().setNeedFlush(true);
+ }
+ AbstractContext context = new Context(nextMessage);
+ this.getFireReceiver().doMessage(nextMessage, context);
+ }
+
+ protected void sendMessage(IMessage msg, List<WindowBaseValue> messages) {
+ if (messages == null || messages.size() == 0) {
+ return;
+ }
+ List<JSONObject> connectMsgs;
+ if (this.expression == null) {
+ List<Map<String, Object>> rows = converToMapFromList(messages);
+ connectMsgs = connectJoin(msg, rows, joinType, rightAsName);
+ } else {
+ List<Map<String, Object>> rows = matchRows(msg.getMessageBody(), converToMapFromList(messages));
+ connectMsgs = connectJoin(msg, rows, joinType, rightAsName);
+ }
+ for (int i = 0; i < connectMsgs.size(); i++) {
+ if (i == connectMsgs.size() - 1) {
+ sendMessage(connectMsgs.get(i), true);
+ } else {
+ sendMessage(connectMsgs.get(i), false);
+ }
+ }
+ }
+
+ @Override
+ public void removeInstanceFromMap(WindowInstance windowInstance) {
+ String begin = DateUtil.getBeforeMinutesTime(windowInstance.getStartTime(), (this.retainWindowCount - 1) * this.sizeInterval);
+ String deletePrefix = MapKeyUtil.createKey(windowInstance.getWindowNameSpace(), windowInstance.getWindowName(), begin);
+ for (Map.Entry<String, WindowInstance> tmp : windowInstanceMap.entrySet()) {
+ if (tmp.getKey().compareToIgnoreCase(deletePrefix) <= 0) {
+ windowInstanceMap.remove(tmp);
+ }
+ }
+ }
+
+ public int getRetainWindowCount() {
+ return retainWindowCount;
+ }
+
+ public void setRetainWindowCount(int retainWindowCount) {
+ this.retainWindowCount = retainWindowCount;
+ }
+
+ public List<String> getLeftJoinFieldNames() {
+ return leftJoinFieldNames;
+ }
+
+ public void setLeftJoinFieldNames(List<String> leftJoinFieldNames) {
+ this.leftJoinFieldNames = leftJoinFieldNames;
+ }
+
+ public List<String> getRightJoinFieldNames() {
+ return rightJoinFieldNames;
+ }
+
+ public void setRightJoinFieldNames(List<String> rightJoinFieldNames) {
+ this.rightJoinFieldNames = rightJoinFieldNames;
+ }
+
+ public String getRightAsName() {
+ return rightAsName;
+ }
+
+ public void setRightAsName(String rightAsName) {
+ this.rightAsName = rightAsName;
+ }
+
+ public String getJoinType() {
+ return joinType;
+ }
+
+ public void setJoinType(String joinType) {
+ this.joinType = joinType;
+ }
+
+ public String getExpression() {
+ return expression;
+ }
+
+ public void setExpression(String expression) {
+ this.expression = expression;
+ }
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/Operator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/Operator.java
new file mode 100644
index 0000000..e38c029
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/Operator.java
@@ -0,0 +1,76 @@
+/*
+ * 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.join;
+
+import com.alibaba.fastjson.JSONObject;
+
+import java.security.MessageDigest;
+import java.util.List;
+
+public interface Operator {
+
+ public static String generateKey(JSONObject messageBody, String joinLabel, List<String> leftJoinFieldNames, List<String> rightJoinFieldNames) {
+ StringBuffer buffer = new StringBuffer();
+ if ("left".equalsIgnoreCase(joinLabel)) {
+ for (String field : leftJoinFieldNames) {
+ String value = messageBody.getString(field);
+ buffer.append(value).append("_");
+ }
+ } else {
+ for (String field : rightJoinFieldNames) {
+ String[] rightFields = field.split(".");
+ if (rightFields.length > 1) {
+ field = rightFields[1];
+ }
+ String value = messageBody.getString(field);
+ buffer.append(value).append("_");
+ }
+ }
+
+ buffer.charAt(buffer.length() - 1);
+
+ return MD5(buffer.toString());
+ }
+
+ public static String MD5(String s) {
+ char hexDigits[] = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'};
+
+ try {
+ byte[] btInput = s.getBytes();
+ // 获得MD5摘要算法的 MessageDigest 对象
+ MessageDigest mdInst = MessageDigest.getInstance("MD5");
+ // 使用指定的字节更新摘要
+ mdInst.update(btInput);
+ // 获得密文
+ byte[] md = mdInst.digest();
+ // 把密文转换成十六进制的字符串形式
+ int j = md.length;
+ char str[] = new char[j * 2];
+ int k = 0;
+ for (int i = 0; i < j; i++) {
+ byte byte0 = md[i];
+ str[k++] = hexDigits[byte0 >>> 4 & 0xf];
+ str[k++] = hexDigits[byte0 & 0xf];
+ }
+ return new String(str);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new RuntimeException(e.getMessage(), e);
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java
new file mode 100644
index 0000000..b1ce2a5
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java
@@ -0,0 +1,321 @@
+/*
+ * 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.shuffle;
+
+import org.apache.rocketmq.streams.common.channel.builder.IShuffleChannelBuilder;
+import org.apache.rocketmq.streams.common.channel.impl.memory.MemoryCache;
+import org.apache.rocketmq.streams.common.channel.impl.memory.MemoryChannel;
+import org.apache.rocketmq.streams.common.channel.impl.memory.MemorySink;
+import org.apache.rocketmq.streams.common.channel.impl.memory.MemorySource;
+import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink;
+import org.apache.rocketmq.streams.common.channel.sink.ISink;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
+import org.apache.rocketmq.streams.common.component.ComponentCreator;
+import org.apache.rocketmq.streams.common.channel.builder.IChannelBuilder;
+import org.apache.rocketmq.streams.common.configurable.IConfigurableIdentification;
+import org.apache.rocketmq.streams.common.interfaces.IStreamOperator;
+import org.apache.rocketmq.streams.common.interfaces.ISystemMessageProcessor;
+import org.apache.rocketmq.streams.common.topology.ChainPipeline;
+import org.apache.rocketmq.streams.common.utils.ReflectUtil;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent;
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.lang.reflect.Field;
+import java.util.*;
+import java.util.concurrent.LinkedBlockingQueue;
+
+public abstract class AbstractSystemChannel implements IConfigurableIdentification, ISystemMessageProcessor, IStreamOperator {
+
+ protected static final Log LOG = LogFactory.getLog(AbstractSystemChannel.class);
+
+ protected static final String CHANNEL_PROPERTY_KEY_PREFIX = "CHANNEL_PROPERTY_KEY_PREFIX";
+ protected static final String CHANNEL_TYPE = "CHANNEL_TYPE";
+
+ protected ISource consumer;
+ protected AbstractSupportShuffleSink producer;
+ protected Map<String, String> channelConfig = new HashMap<>();
+ ;
+ protected boolean hasCreateShuffleChannel = false;
+
+ public void startChannel() {
+ if (consumer == null) {
+ return;
+ }
+ final AbstractSystemChannel channel = this;
+ consumer.start(this);
+ }
+
+ /**
+ * 如果用户未配置shuffle channel,根据pipline数据源动态创建
+ *
+ * @param pipeline
+ */
+ public void autoCreateShuffleChannel(ChainPipeline pipeline) {
+ if (!hasCreateShuffleChannel) {
+ synchronized (this) {
+ if (!hasCreateShuffleChannel) {
+ ISource piplineSource = pipeline.getSource();
+ ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(
+ IChannelBuilder.class.getName(), ServiceLoaderComponent.class);
+ IChannelBuilder builder = (IChannelBuilder)serviceLoaderComponent.loadService(piplineSource.getClass().getSimpleName());
+ if (builder == null) {
+ throw new RuntimeException("can not create shuffle channel, not find channel builder " + piplineSource.toJson());
+ }
+ if (!IShuffleChannelBuilder.class.isInstance(builder)) {
+ throw new RuntimeException("can not create shuffle channel, builder not imp IShuffleChannelBuilder " + piplineSource.toJson());
+ }
+ IShuffleChannelBuilder shuffleChannelBuilder = (IShuffleChannelBuilder)builder;
+ ISink sink = shuffleChannelBuilder.createBySource(piplineSource);
+ if (!MemoryChannel.class.isInstance(sink) && !AbstractSupportShuffleSink.class.isInstance(sink)) {
+ throw new RuntimeException("can not create shuffle channel, sink not extends AbstractSupportShuffleSink " + piplineSource.toJson());
+ }
+ ISource source = null;
+ if (MemoryChannel.class.isInstance(sink)) {
+ MemoryCache memoryCache = new MemoryCache();
+ memoryCache.setNameSpace(createShuffleChannelNameSpace(pipeline));
+ memoryCache.setConfigureName(createShuffleChannelName(pipeline));
+
+ sink = new MemorySink();
+ source = new MemorySource();
+ ((MemorySink)sink).setMemoryCache(memoryCache);
+ ((MemorySource)source).setMemoryCache(memoryCache);
+ memoryCache.init();
+ }
+
+ Properties properties = new Properties();
+ putDynamicPropertyValue(new HashSet<>(), properties);
+
+ AbstractSupportShuffleSink shuffleSink = (AbstractSupportShuffleSink)sink;
+ shuffleSink.setSplitNum(getShuffleSplitCount(shuffleSink));
+ shuffleSink.setNameSpace(createShuffleChannelNameSpace(pipeline));
+ shuffleSink.setConfigureName(createShuffleChannelName(pipeline));
+ String topicFiledName = shuffleSink.getShuffleTopicFieldName();
+ String shuffleTopic = null;
+ //内存模式,是无topic的
+ if (StringUtil.isNotEmpty(topicFiledName)) {
+ String topic = ReflectUtil.getDeclaredField(shuffleSink, topicFiledName);
+ shuffleTopic = createShuffleTopic(topic, pipeline);
+ ReflectUtil.setBeanFieldValue(shuffleSink, topicFiledName, shuffleTopic);
+ }
+
+ //修改和window有关的属性,如groupname,tags
+ List<Field> fields = ReflectUtil.getDeclaredFieldsContainsParentClass(sink.getClass());
+ for (Field field : fields) {
+ String fieldName = field.getName();
+ String value = properties.getProperty(fieldName);
+ if (StringUtil.isNotEmpty(value)) {
+ ReflectUtil.setBeanFieldValue(sink, fieldName, value);
+ }
+ }
+
+ shuffleSink.init();//在这里完成shuffle channel的创建
+ if (source == null) {
+ source = shuffleChannelBuilder.copy(piplineSource);
+ }
+
+ //修改和window有关的属性,如groupname,tags
+
+ fields = ReflectUtil.getDeclaredFieldsContainsParentClass(source.getClass());
+ for (Field field : fields) {
+ String fieldName = field.getName();
+ String value = properties.getProperty(fieldName);
+ if (StringUtil.isNotEmpty(value)) {
+ ReflectUtil.setBeanFieldValue(source, fieldName, value);
+ }
+ }
+
+ source.setNameSpace(sink.getNameSpace());
+ source.setConfigureName(sink.getConfigureName());
+ //修改主题
+ if (shuffleTopic != null && topicFiledName != null) {
+ ReflectUtil.setBeanFieldValue(source, topicFiledName, shuffleTopic);
+ }
+
+ source.init();
+
+ this.producer = shuffleSink;
+ this.consumer = source;
+ }
+ }
+ }
+ }
+
+ /**
+ * 根据数据源的名字,给shufflechannel取主题名
+ *
+ * @param topic
+ * @param message
+ * @return
+ */
+ protected abstract String createShuffleTopic(String topic,
+ ChainPipeline message);
+
+ protected abstract int getShuffleSplitCount(AbstractSupportShuffleSink shuffleSink);
+
+ /**
+ * shuffle channel的名字
+ *
+ * @param message
+ * @return
+ */
+ protected abstract String createShuffleChannelName(ChainPipeline message);
+
+ /**
+ * shuffle channel的名字
+ *
+ * @param message
+ * @return
+ */
+ protected abstract String createShuffleChannelNameSpace(ChainPipeline message);
+
+ protected Map<String, String> getChannelConfig() {
+ return channelConfig;
+ }
+
+ protected abstract String getDynamicPropertyValue();
+
+ /**
+ * 创建channel,根据配置文件配置channel的连接信息
+ *
+ * @return
+ */
+ protected ISource createSource(String namespace, String name) {
+
+ IChannelBuilder builder = createBuilder();
+ if (builder == null) {
+ return null;
+ }
+ Properties properties = createChannelProperties(namespace);
+ ISource source = builder.createSource(namespace, name, properties, null);
+ source.init();
+ return source;
+ }
+
+ /**
+ * 创建channel,根据配置文件配置channel的连接信息
+ *
+ * @return
+ */
+ protected AbstractSupportShuffleSink createSink(String namespace, String name) {
+
+ IChannelBuilder builder = createBuilder();
+ if (builder == null) {
+ return null;
+ }
+ Properties properties = createChannelProperties(namespace);
+
+ ISink sink = builder.createSink(namespace, name, properties, null);
+ if (!AbstractSupportShuffleSink.class.isInstance(sink)) {
+ throw new RuntimeException("can not support shuffle " + sink.toJson());
+ }
+ AbstractSupportShuffleSink abstractSupportShuffleSink = (AbstractSupportShuffleSink)sink;
+ abstractSupportShuffleSink.init();
+ return abstractSupportShuffleSink;
+ }
+
+ /**
+ * create channel builder
+ *
+ * @return
+ */
+ protected IChannelBuilder createBuilder() {
+ String type = ComponentCreator.getProperties().getProperty(getChannelConfig().get(CHANNEL_TYPE));
+ if (StringUtil.isEmpty(type)) {
+ return null;
+ }
+ ServiceLoaderComponent serviceLoaderComponent = ComponentCreator.getComponent(
+ IChannelBuilder.class.getName(), ServiceLoaderComponent.class);
+ IChannelBuilder builder = (IChannelBuilder)serviceLoaderComponent.loadService(type);
+ return builder;
+ }
+
+ /**
+ * 根据属性文件配置
+ *
+ * @return
+ */
+ protected Properties createChannelProperties(String namespace) {
+ Properties properties = new Properties();
+ Iterator<Map.Entry<Object, Object>> it = ComponentCreator.getProperties().entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry<Object, Object> entry = it.next();
+ String key = (String)entry.getKey();
+ String value = (String)entry.getValue();
+ if (key.startsWith(getChannelConfig().get(CHANNEL_PROPERTY_KEY_PREFIX))) {
+ String channelKey = key.replace(getChannelConfig().get(CHANNEL_PROPERTY_KEY_PREFIX), "");
+ if (channelKey.startsWith(namespace)) {//支持基于namespace 做shuffle window共享
+ channelKey = channelKey.replace(namespace, "");
+ properties.put(channelKey, value);
+ } else {
+ if (!properties.containsKey(channelKey)) {
+ properties.put(channelKey, value);
+ }
+ }
+
+ }
+
+ }
+ Set<String> mutilPropertySet = new HashSet<>();
+ String dynamicProperty = properties.getProperty("dynamic.property");
+ if (dynamicProperty != null) {
+
+ String dynamicPropertyValue = getDynamicPropertyValue();
+ String[] mutilPropertys = dynamicProperty.split(",");
+
+ for (String properyKey : mutilPropertys) {
+ properties.put(properyKey, dynamicPropertyValue);
+ mutilPropertySet.add(properyKey);
+ }
+
+ }
+ putDynamicPropertyValue(mutilPropertySet, properties);
+ return properties;
+ }
+
+ /**
+ * 如果需要额外的动态属性,可以在子类添加
+ *
+ * @param dynamiPropertySet
+ */
+ protected void putDynamicPropertyValue(Set<String> dynamiPropertySet, Properties properties) {
+
+ }
+
+ public ISource getConsumer() {
+ return consumer;
+ }
+
+ public ISink getProducer() {
+ return producer;
+ }
+
+ public void sendMessage(IMessage message) {
+ List<IMessage> msgs = new ArrayList<>();
+ msgs.add(message);
+ producer.batchSave(msgs);
+ producer.flush();
+ }
+
+ //public void flush() {
+ // producer.flush();
+ //}
+
+}
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
new file mode 100644
index 0000000..184b8b4
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java
@@ -0,0 +1,542 @@
+/*
+ * 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.shuffle;
+
+import com.alibaba.fastjson.JSONArray;
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink;
+import org.apache.rocketmq.streams.common.channel.source.AbstractSource;
+import org.apache.rocketmq.streams.common.channel.source.systemmsg.NewSplitMessage;
+import org.apache.rocketmq.streams.common.channel.source.systemmsg.RemoveSplitMessage;
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.checkpoint.CheckPointMessage;
+import org.apache.rocketmq.streams.common.checkpoint.CheckPointState;
+import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
+import org.apache.rocketmq.streams.common.interfaces.ISystemMessage;
+import org.apache.rocketmq.streams.common.topology.ChainPipeline;
+import org.apache.rocketmq.streams.common.topology.model.Pipeline;
+import org.apache.rocketmq.streams.common.utils.CollectionUtil;
+import org.apache.rocketmq.streams.common.utils.DateUtil;
+import org.apache.rocketmq.streams.common.utils.TraceUtil;
+import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
+import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow;
+import org.apache.rocketmq.streams.window.operator.AbstractWindow;
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+import org.apache.rocketmq.streams.window.model.WindowInstance;
+import org.apache.rocketmq.streams.window.model.WindowCache;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.window.operator.impl.WindowOperator.WindowRowOperator;
+import org.apache.rocketmq.streams.window.source.WindowRireSource;
+import org.apache.rocketmq.streams.window.storage.ShufflePartitionManager;
+
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 负责处理分片
+ */
+public class ShuffleChannel extends AbstractSystemChannel {
+
+ protected static final Log LOG = LogFactory.getLog(ShuffleChannel.class);
+
+ protected static final String SHUFFLE_QUEUE_ID = "SHUFFLE_QUEUE_ID";
+ private static final String SHUFFLE_KEY = "SHUFFLE_KEY";
+ private static final String PROCESS_NAME = "PROCESS_NAME";
+
+ protected static final String SHUFFLE_MESSAGES = "SHUFFLE_MESSAGES";
+ protected String MSG_OWNER = "MSG_OWNER";//消息所属的window
+
+ protected String WINDOW_INSTANCE_START_TIME = "_start_time";
+ protected String WINDOW_INSTANCE_END_TIME = "_end_time";
+
+ private static final String SHUFFLE_TRACE_ID = "SHUFFLE_TRACE_ID";
+
+ protected ShuffleOutputDataSource shuffleSink;
+
+ protected Map<String, ISplit> queueMap = new ConcurrentHashMap<>();
+ protected List<ISplit> queueList;//所有的分片
+
+ // protected NotifyChannel notfiyChannel;//负责做分片的通知管理
+ protected AbstractShuffleWindow window;
+ private Set<String> currentQueueIds;//当前管理的分片
+
+ protected transient WindowRireSource windowRireSource;
+
+ public ShuffleChannel(AbstractShuffleWindow window) {
+ this.window = window;
+ channelConfig = new HashMap<>();
+ channelConfig.put(CHANNEL_PROPERTY_KEY_PREFIX, ConfigureFileKey.WINDOW_SHUFFLE_CHANNEL_PROPERTY_PREFIX);
+ channelConfig.put(CHANNEL_TYPE, ConfigureFileKey.WINDOW_SHUFFLE_CHANNEL_TYPE);
+ this.consumer = createSource(window.getNameSpace(),window.getConfigureName());
+
+ this.producer = createSink(window.getNameSpace(),window.getConfigureName());
+ if(this.consumer==null||this.producer==null){
+ autoCreateShuffleChannel(window.getFireReceiver().getPipeline());
+ }
+ if(this.consumer instanceof AbstractSource){
+ ((AbstractSource)this.consumer).setJsonData(true);
+ }
+ this.shuffleSink = createWindowTaskOutputDataSource();
+ this.shuffleSink.openAutoFlush();
+ //this.notfiyChannel = NotifyChannel.getInstance();
+ //if(this.notfiyChannel.producer==null||this.notfiyChannel.consumer==null){
+ // this.notfiyChannel.autoCreateShuffleChannel(window.getFireReceiver().getPipeline());
+ //}
+ //this.notfiyChannel.startChannel();//启动通知管理,里面做了重入,最终只启动一个
+ if (producer!=null&&(queueList == null || queueList.size() == 0) ){
+ queueList = producer.getSplitList();
+ Map<String, ISplit> tmp = new ConcurrentHashMap<>();
+ for (ISplit queue : queueList) {
+ tmp.put(queue.getQueueId(), queue);
+ }
+
+ this.queueMap = tmp;
+ }
+ }
+
+ @Override
+ protected String getDynamicPropertyValue() {
+ String dynamicPropertyValue= MapKeyUtil.createKey(window.getNameSpace(),window.getConfigureName());
+ dynamicPropertyValue = dynamicPropertyValue.replaceAll("\\.", "_").replaceAll(";","_");
+ return dynamicPropertyValue;
+ }
+
+ @Override
+ protected int getShuffleSplitCount(AbstractSupportShuffleSink shuffleSink) {
+ int splitNum=shuffleSink.getSplitNum();
+ return splitNum>0?splitNum:32;
+ }
+
+ /**
+ * 接收到分片信息,如果是系统消息,做缓存刷新,否则把消息放入缓存,同时计算存储的有效性
+ *
+ * @param oriMessage
+ * @param context
+ * @return
+ */
+
+
+ @Override
+ public Object doMessage(IMessage oriMessage, AbstractContext context) {
+ if (oriMessage.getHeader().isSystemMessage()) {
+ ISystemMessage systemMessage=oriMessage.getSystemMessage();
+ if(systemMessage instanceof CheckPointMessage){
+ this.checkpoint(oriMessage, context,(CheckPointMessage)systemMessage);
+ }else if(systemMessage instanceof NewSplitMessage){
+ this.addNewSplit(oriMessage,context,(NewSplitMessage)systemMessage);
+ }else if(systemMessage instanceof RemoveSplitMessage){
+ this.removeSplit(oriMessage,context,(RemoveSplitMessage)systemMessage);
+ }else {
+ throw new RuntimeException("can not support this system message "+systemMessage.getClass().getName());
+ }
+ afterFlushCallback(oriMessage,context);
+ return null;
+
+ }
+ /**
+ * 过滤不是这个window的消息,一个shuffle通道,可能多个window共享,这里过滤掉非本window的消息
+ */
+ boolean isFilter=filterNotOwnerMessage(oriMessage);
+ if(isFilter){
+ return null;
+ }
+ String queueId=oriMessage.getHeader().getQueueId();
+ //ISplit channelQueue=queueMap.get(queueId);
+ //boolean containQueueId=notfiyChannel.contains(queueId);
+ //notfiyChannel.dealMessageQueue(channelQueue);
+
+ JSONArray messages = oriMessage.getMessageBody().getJSONArray(SHUFFLE_MESSAGES);
+ if(messages==null){
+ return null;
+ }
+
+ String traceId = oriMessage.getMessageBody().getString(SHUFFLE_TRACE_ID);
+ if (!StringUtil.isEmpty(traceId)) {
+ TraceUtil.debug(traceId, "shuffle message in", "received message size:" + messages.size());
+ }
+
+ for (Object obj: messages) {
+ IMessage message = new Message((JSONObject) obj);
+ message.getHeader().setQueueId(queueId);
+
+ List<WindowInstance> windowInstances=window.queryOrCreateWindowInstance(message,queueId);
+ if(windowInstances==null||windowInstances.size()==0){
+ continue;
+ }
+ for(WindowInstance windowInstance:windowInstances){
+ String windowInstanceId = windowInstance.createWindowInstanceId();
+ if(!window.getWindowInstanceMap().containsKey(windowInstanceId)){
+ window.getWindowInstanceMap().putIfAbsent(windowInstanceId,windowInstance);
+ synchronized (this){
+ if(window.getFireMode()==2){
+ //这个模式窗口触发不会清理数据,需要额外的创建一个实例做最后的存储清理
+ Date endTime=DateUtil.parseTime(windowInstance.getEndTime());
+ Date lastFireTime=DateUtil.addDate(TimeUnit.SECONDS,endTime,window.getWaterMarkMinute()*window.getTimeUnitAdjust());
+ WindowInstance lastClearWindowInstance=window.createWindowInstance(windowInstance.getStartTime(),windowInstance.getEndTime(),DateUtil.format(lastFireTime),queueId);
+
+ addNeedFlushWindowInstance(lastClearWindowInstance);
+ }
+ }
+
+ }
+ if(windowInstance.isNewWindowInstance()){
+ addNeedFlushWindowInstance(windowInstance);
+ windowInstance.setNewWindowInstance(false);
+ ShufflePartitionManager.getInstance().setWindowInstanceFinished(windowInstance.createWindowInstanceId());
+ }
+ }
+
+ message.getMessageBody().put(WindowInstance.class.getSimpleName(), windowInstances);
+ message.getMessageBody().put(AbstractWindow.class.getSimpleName(), window);
+ beforeBatchAdd(oriMessage,message);
+ shuffleSink.batchAdd(message);
+ for(WindowInstance windowInstance:windowInstances){
+ window.getWindowFireSource().updateWindowInstanceLastUpdateTime(windowInstance,window.updateMaxEventTime(message));
+ }
+ }
+
+ return null;
+ }
+
+ @Override
+ protected Map<String, String> getChannelConfig() {
+ return channelConfig;
+ }
+
+ @Override
+ protected void putDynamicPropertyValue(Set<String> dynamiPropertySet,Properties properties){
+ String groupName="groupName";
+ if(!dynamiPropertySet.contains(groupName)){
+ properties.put(groupName,getDynamicPropertyValue());
+ }
+ if(!dynamiPropertySet.contains("tags")){
+ properties.put("tags",getDynamicPropertyValue());
+ }
+ }
+
+ /**
+ * 1个pipeline一个 shuffle topic
+ *
+ * @param topic
+ * @param pipeline
+ * @return
+ */
+ @Override
+ protected String createShuffleTopic(String topic, ChainPipeline pipeline) {
+ return "shuffle_" + topic + "_" + pipeline.getSource().getNameSpace().replaceAll("\\.", "_") + "_" + pipeline
+ .getConfigureName().replaceAll("\\.", "_").replaceAll(";", "_");
+ }
+
+ /**
+ * 一个window 一个channel
+ *
+ * @param pipeline
+ * @return
+ */
+ @Override
+ protected String createShuffleChannelName(ChainPipeline pipeline) {
+ return getDynamicPropertyValue();
+ }
+
+ /**
+ * 和pipeline namespace 相同
+ *
+ * @param pipeline
+ * @return
+ */
+ @Override
+ protected String createShuffleChannelNameSpace(ChainPipeline pipeline) {
+ return pipeline.getSource().getNameSpace();
+ }
+
+ /**
+ * 对于接收到新的window task消息后的处理器
+ *
+ * @return
+ */
+ protected ShuffleOutputDataSource createWindowTaskOutputDataSource() {
+
+ ShuffleOutputDataSource outputDataSource =new ShuffleOutputDataSource();
+ outputDataSource.init();
+ outputDataSource.setConfigureName("shuffleChannel");
+ return outputDataSource;
+ }
+
+ @Override
+ public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) {
+ this.currentQueueIds=newSplitMessage.getCurrentSplitIds();
+ List<WindowInstance> allWindowInstances=WindowInstance.queryAllWindowInstance(DateUtil.getCurrentTimeString(),window,newSplitMessage.getSplitIds());
+ if(CollectionUtil.isNotEmpty(allWindowInstances)){
+ Set<String> partitionNumKeys=new HashSet<>();
+ for(WindowInstance windowInstance:allWindowInstances){
+ windowInstance.setNewWindowInstance(false);
+ window.getWindowInstanceMap().putIfAbsent(windowInstance.createWindowInstanceId(),windowInstance);
+ window.getWindowFireSource().registFireWindowInstanceIfNotExist(windowInstance,window);
+ for(String queueId:newSplitMessage.getSplitIds()){
+ String key=window.getWindowMaxValueManager().createSplitNumberKey(windowInstance,queueId);
+ partitionNumKeys.add(key);
+ window.getStorage().loadSplitData2Local(queueId,windowInstance.createWindowInstanceId(),window.getWindowBaseValueClass(),new WindowRowOperator(windowInstance,queueId,window));
+ }
+ }
+ window.getWindowMaxValueManager().loadMaxSplitNum(partitionNumKeys);
+
+ }else {
+ for(String queueId:newSplitMessage.getSplitIds()){
+ ShufflePartitionManager.getInstance().setSplitFinished(queueId);
+ }
+ }
+
+ }
+
+ @Override
+ public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) {
+ this.currentQueueIds=removeSplitMessage.getCurrentSplitIds();
+ Set<String> queueIds=removeSplitMessage.getSplitIds();
+ if(queueIds!=null){
+ for(String queueId:queueIds){
+ ShufflePartitionManager.getInstance().setSplitInValidate(queueId);
+ window.clearCache(queueId);
+
+ }
+ window.getWindowMaxValueManager().removeKeyPrefixFromLocalCache(queueIds);
+ //window.getWindowFireSource().removeSplit(queueIds);
+ }
+ }
+
+ @Override
+ public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) {
+ if(message.getHeader().isNeedFlush()){
+ this.flush(message.getHeader().getCheckpointQueueIds());
+ }
+ CheckPointState checkPointState= new CheckPointState();
+ checkPointState.setQueueIdAndOffset(this.shuffleSink.getFinishedQueueIdAndOffsets(checkPointMessage));
+ checkPointMessage.reply(checkPointState);
+ }
+
+ @Override
+ public String getConfigureName() {
+ return window.getConfigureName()+"_shuffle";
+ }
+
+ @Override
+ public String getNameSpace() {
+ return window.getNameSpace();
+ }
+
+ @Override
+ public String getType() {
+ return Pipeline.TYPE;
+ }
+
+ protected class ShuffleOutputDataSource extends WindowCache {
+ protected List<WindowInstance> notSaveWindowInstances=new ArrayList<>();//未保存的windowinstance
+
+ public ShuffleOutputDataSource() {
+ }
+
+ @Override
+ protected boolean batchInsert(List<IMessage> messageList) {
+ Map<Pair<String, String>, List<IMessage>> instance2Messages = new HashMap<>();
+ Map<String, WindowInstance> windowInstanceMap = new HashMap<>();
+ groupByWindowInstanceAndQueueId(messageList, instance2Messages, windowInstanceMap);
+ Iterator<Map.Entry<Pair<String, String>, List<IMessage>>> it = instance2Messages.entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry<Pair<String, String>, List<IMessage>> entry = it.next();
+ Pair<String, String> queueIdAndInstanceKey = entry.getKey();
+ List<IMessage> messages = entry.getValue();
+ WindowInstance windowInstance = windowInstanceMap.get(queueIdAndInstanceKey.getRight());
+ window.shuffleCalculate(messages, windowInstance, queueIdAndInstanceKey.getLeft());
+ }
+ return true;
+ }
+
+ @Override
+ public boolean flushMessage(List<IMessage> messages) {
+ saveOtherState();
+ return super.flushMessage(messages);
+ }
+
+ protected void saveOtherState(){
+ if(notSaveWindowInstances.size()>0){
+ if(notSaveWindowInstances.size()>0){
+ List<WindowInstance> copy=null;
+ synchronized (this){
+ copy=this.notSaveWindowInstances;
+ this.notSaveWindowInstances=new ArrayList<>();
+ }
+ Set<String> existWindowInstaceIds=new HashSet<>();
+ List<WindowInstance> windowInstances=new ArrayList<>();
+ for(WindowInstance windowInstance:copy){
+ String windowInstanceId=windowInstance.createWindowInstanceId();
+ if(existWindowInstaceIds.contains(windowInstanceId)){
+ continue;
+ }
+ windowInstances.add(windowInstance);
+ existWindowInstaceIds.add(windowInstanceId);
+ }
+ ORMUtil.batchReplaceInto(windowInstances);
+ }
+
+ }
+ window.getWindowMaxValueManager().flush();
+ }
+
+ @Override
+ protected String generateShuffleKey(IMessage message) {
+ return null;
+ }
+ }
+
+ /**
+ * 根据message,把message分组到不同的group,分别处理
+ *
+ * @param messageList
+ * @param instance2Messages
+ * @param windowInstanceMap
+ */
+ protected void groupByWindowInstanceAndQueueId(List<IMessage> messageList, Map<Pair<String, String>, List<IMessage>> instance2Messages,
+ Map<String, WindowInstance> windowInstanceMap) {
+ for (IMessage message : messageList) {
+
+ List<WindowInstance> windowInstances = (List<WindowInstance>)message.getMessageBody().get(WindowInstance.class.getSimpleName());
+ String queueId = message.getHeader().getQueueId();
+ for(WindowInstance windowInstance:windowInstances){
+ String windowInstanceId = windowInstance.createWindowInstanceId();
+ Pair<String, String> queueIdAndInstanceKey = Pair.of(queueId, windowInstanceId);
+ List<IMessage> messages = instance2Messages.get(queueIdAndInstanceKey);
+ if (messages == null) {
+ messages = new ArrayList<>();
+ instance2Messages.put(queueIdAndInstanceKey, messages);
+ }
+ messages.add(message);
+ windowInstanceMap.put(windowInstanceId, windowInstance);
+ }
+
+ String oriQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID);
+ String oriOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET);
+ message.getHeader().setQueueId(oriQueueId);
+ message.getHeader().setOffset(oriOffset);
+
+ }
+ }
+
+ public ISplit getSplit(Integer index){
+ return queueList.get(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(";");
+ }
+ }
+ msg.put(SHUFFLE_TRACE_ID, traceIds);
+ TraceUtil.debug(traceIds.toString(), "origin message out", split.getQueueId());
+
+ return msg;
+ }
+
+ public ISplit getChannelQueue(String key){
+ int index=hash(key);
+ ISplit targetQueue = queueList.get(index);
+ return targetQueue;
+ }
+
+ public int hash(Object key) {
+ int mValue=queueList.size();
+ int h = 0;
+ if (key != null) {
+ h = key.hashCode() ^ (h >>> 16);
+ if (h < 0) {
+ h = -h;
+ }
+ }
+ return h % mValue;
+ }
+
+ public void flush(Set<String> checkpointQueueIds) {
+ shuffleSink.flush(checkpointQueueIds);
+ }
+
+ /**
+ * 每次checkpoint的回调函数,默认是空实现,子类可以扩展实现
+ *
+ * @param oriMessage
+ * @param context
+ */
+ protected void afterFlushCallback(IMessage oriMessage, AbstractContext context) {
+ }
+
+ /**
+ * shuffle 获取数据,插入缓存前的回调函数,默认空实现,可以子类覆盖扩展
+ *
+ * @param oriMessage
+ * @param message
+ */
+ protected void beforeBatchAdd(IMessage oriMessage, IMessage message) {
+ }
+
+ /**
+ * 过滤掉不是这个window的消息
+ *
+ * @param oriMessage
+ * @return
+ */
+ protected boolean filterNotOwnerMessage(IMessage oriMessage) {
+ String owner=oriMessage.getMessageBody().getString(MSG_OWNER);
+ if(owner!=null&&owner.equals(getDynamicPropertyValue())){
+ return false;
+ }
+ return true;
+ }
+
+ public Set<String> getCurrentQueueIds() {
+ return currentQueueIds;
+ }
+
+ public List<ISplit> getQueueList() {
+ return queueList;
+ }
+
+ public synchronized void addNeedFlushWindowInstance(WindowInstance windowInstance){
+ if(!window.isLocalStorageOnly()){
+ this.shuffleSink.notSaveWindowInstances.add(windowInstance);
+ }
+ }
+
+ public synchronized void clearCache(WindowInstance windowInstance){
+ this.shuffleSink.notSaveWindowInstances.remove(windowInstance);
+ }
+
+}
\ No newline at end of file
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/WindowRireSource.java
new file mode 100644
index 0000000..c6495db
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/source/WindowRireSource.java
@@ -0,0 +1,336 @@
+/*
+ * 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.source;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageCache;
+import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack;
+import org.apache.rocketmq.streams.common.channel.sinkcache.impl.AbstractMutilSplitMessageCache;
+import org.apache.rocketmq.streams.common.channel.source.AbstractSupportOffsetResetSource;
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.interfaces.IStreamOperator;
+import org.apache.rocketmq.streams.window.model.WindowInstance;
+import org.apache.rocketmq.streams.window.operator.AbstractWindow;
+
+public class WindowRireSource extends AbstractSupportOffsetResetSource implements IStreamOperator {
+ protected static final Log LOG = LogFactory.getLog(WindowRireSource.class);
+ private AbstractWindow window;
+ protected transient ConcurrentHashMap<String,Long> maxEventTimes=new ConcurrentHashMap();//max event time procced by window
+ protected transient ConcurrentHashMap<String,Long> eventTimeLastUpdateTimes=new ConcurrentHashMap<>();
+ protected transient ScheduledExecutorService fireCheckScheduler;//检查是否触发
+ protected transient ScheduledExecutorService checkpointScheduler;
+ protected transient ConcurrentHashMap<String,WindowInstance> windowInstances=new ConcurrentHashMap();
+
+ protected transient IMessageCache<WindowInstance> fireInstanceCache=new WindowInstanceCache();
+ //正在触发中的windowintance
+ protected transient ConcurrentHashMap<String,WindowInstance> firingWindowInstances=new ConcurrentHashMap<>();
+
+
+ //<windowinstanceId,<queueId,offset>>
+ protected transient ConcurrentHashMap<String,Map<String,String>> windowInstanceQueueOffsets=new ConcurrentHashMap<>();
+
+ public WindowRireSource(AbstractWindow window){
+ this.window=window;
+ }
+
+ @Override
+ protected boolean initConfigurable() {
+ fireCheckScheduler=new ScheduledThreadPoolExecutor(2);
+ checkpointScheduler=new ScheduledThreadPoolExecutor(3);
+ setReceiver(window.getFireReceiver());
+ fireInstanceCache.openAutoFlush();
+ return super.initConfigurable();
+ }
+ @Override
+ protected boolean startSource() {
+ //检查window instance,如果已经到了触发时间,且符合触发条件,直接触发,如果到了触发时间,还未符合触发条件。则放入触发列表。下次调度时间是下一个最近触发的时间
+
+ fireCheckScheduler.scheduleWithFixedDelay(new Runnable() {
+ // long startTime=System.currentTimeMillis();
+ @Override
+ public void run() {
+ try {
+ //System.out.println("fire schdule time is "+(System.currentTimeMillis()-startTime)+" windowinstance count is "+windowInstances.size());
+ //startTime=System.currentTimeMillis();
+ if(windowInstances.size()==0){
+ //if(eventTimeLastUpdateTime!=null){
+ // int gap=(int)(System.currentTimeMillis()-eventTimeLastUpdateTime);
+ // if(window.getMsgMaxGapSecond()!=null&&gap>window.getMsgMaxGapSecond()*1000){
+ // for(String key:fireCounts.keySet()){
+ // Integer count=fireCounts.get(key);
+ // if(count==0){
+ // System.out.println("===================== "+key+":"+count);
+ // }
+ // }
+ // }
+ //}
+
+ }
+ List<WindowInstance> windowInstanceList=new ArrayList<>();
+ windowInstanceList.addAll(windowInstances.values());
+ long fireStartTime=System.currentTimeMillis();
+ Collections.sort(windowInstanceList, new Comparator<WindowInstance>() {
+ @Override
+ public int compare(WindowInstance o1, WindowInstance o2) {
+ int value= o1.getFireTime().compareTo(o2.getFireTime());
+ if(value!=0){
+ return value;
+ }
+ return o2.getStartTime().compareTo(o1.getStartTime());
+ }
+ });
+ for(int i=0;i<windowInstanceList.size();i++){
+ WindowInstance windowInstance = windowInstanceList.get(i);
+
+ boolean success= executeFireTask(windowInstance,false);
+ if(!success){
+ continue;
+ }
+ windowInstances.remove(windowInstance.createWindowInstanceId());
+
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ },0,1, TimeUnit.SECONDS);
+
+ //定时发送checkpoint,提交和保存数据。在pull模式会有用
+ //fireCheckScheduler.scheduleWithFixedDelay(new Runnable() {
+ //
+ // @Override
+ // public void run() {
+ // if(checkPointManager.getCurrentSplits()==null||checkPointManager.getCurrentSplits().size()==0){
+ // return;
+ // }
+ // for(WindowInstance windowInstance:firingWindowInstances.values()){
+ // Set<String> splits=checkPointManager.getCurrentSplits();
+ // Set<String> windowInstanceSplits=new HashSet<>();
+ // for(String splitId:splits){
+ // //String windowInstanceSpiltId=windowInstance.createWindowInstancePartitionId();
+ // windowInstanceSplits.add(splitId);
+ // }
+ // sendCheckpoint(windowInstanceSplits);
+ // }
+ //
+ // }
+ //},0,getCheckpointTime(), TimeUnit.MILLISECONDS);
+
+ return false;
+ }
+
+
+
+ /**
+ * 如果没有window instance,则注册,否则放弃
+ * @param windowInstance
+ */
+ public void registFireWindowInstanceIfNotExist(WindowInstance windowInstance, AbstractWindow window){
+ String windowInstanceId=windowInstance.createWindowInstanceId();
+ WindowInstance old= windowInstances.putIfAbsent(windowInstanceId,windowInstance);
+ if(old==null){
+ window.getWindowInstanceMap().put(windowInstanceId,windowInstance);
+ }
+ LOG.debug("register window instance into manager, instance key: " + windowInstance.createWindowInstanceId());
+ }
+ /**
+ * 注册一个window instance
+ * @param windowInstance
+ */
+ public void updateWindowInstanceLastUpdateTime(WindowInstance windowInstance,Long windowMaxEventTime){
+ String windowInstanceId=windowInstance.createWindowInstanceId();
+ if(windowMaxEventTime!=null){
+ this.maxEventTimes.put(windowInstanceId,windowMaxEventTime);
+ this.eventTimeLastUpdateTimes.put(windowInstanceId,System.currentTimeMillis());
+ }
+
+
+ windowInstances.putIfAbsent(windowInstanceId,windowInstance);
+
+
+
+ }
+ /**
+ * 触发窗口
+ * @param windowInstance
+ */
+ public boolean executeFireTask(WindowInstance windowInstance,boolean startNow) {
+ String windowInstanceId=windowInstance.createWindowInstanceId();
+ if (canFire(windowInstance)) {
+ //maybe firimg
+ if (firingWindowInstances.containsKey(windowInstanceId)) {
+ //System.out.println("has firing");
+
+ return true;
+ }
+ //maybe fired
+ //if (!window.getWindowInstanceMap().containsKey(windowInstanceId)) {
+ // return true;
+ //}
+ //start firing
+ firingWindowInstances.put(windowInstanceId, windowInstance);
+ if(startNow){
+ fireWindowInstance(windowInstance);
+ }else {
+ fireInstanceCache.addCache(windowInstance);
+ }
+ return true;
+ }
+ return false;
+ }
+ /**
+ * 触发窗口
+ * @param windowInstance
+ */
+ protected void fireWindowInstance(WindowInstance windowInstance) {
+ try {
+ if (windowInstance == null) {
+ LOG.error("window instance is null!");
+ return;
+ }
+ String windowInstanceId = windowInstance.createWindowInstanceId();
+ if (window == null) {
+ LOG.error(windowInstanceId + "'s window object have been removed!");
+ return;
+ }
+
+
+ if(windowInstance.getLastMaxUpdateTime()==null){
+ windowInstance.setLastMaxUpdateTime(this.maxEventTimes.get(windowInstanceId));
+ }
+ int fireCount=window.fireWindowInstance(windowInstance,windowInstanceQueueOffsets.get(windowInstanceId));
+ LOG.debug("fire instance("+windowInstance.createWindowInstanceId()+" fire count is "+fireCount);
+ firingWindowInstances.remove(windowInstanceId);
+ this.eventTimeLastUpdateTimes.remove(windowInstanceId);
+ this.maxEventTimes.remove(windowInstanceId);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+
+ /**
+ * 是否符合触发条件
+ * @param windowInstance
+ * @return
+ */
+ protected boolean canFire(WindowInstance windowInstance) {
+ String windowInstanceId=windowInstance.createWindowInstanceId();
+ if(window == null){
+ LOG.warn(windowInstanceId + " can't find window!");
+ return false;
+ }
+ Date realFireTime=window.getRealFireTime(windowInstance);
+ /**
+ * 未到触发时间
+ */
+ Long maxEventTime=this.maxEventTimes.get(windowInstanceId);
+ if(maxEventTime==null){
+ return false;
+ }
+ if(maxEventTime-realFireTime.getTime()>=0){
+ return true;
+ }
+ if(maxEventTime-realFireTime.getTime()<0){
+ Long eventTimeLastUpdateTime=this.eventTimeLastUpdateTimes.get(windowInstanceId);
+ if(eventTimeLastUpdateTime==null){
+ return false;
+ }
+ 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 "+windowInstanceId);
+ return true;
+ }
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public Object doMessage(IMessage message, AbstractContext context) {
+ return null;
+ }
+
+
+ protected class WindowInstanceCache extends AbstractMutilSplitMessageCache<WindowInstance>{
+
+ public WindowInstanceCache() {
+ super(new IMessageFlushCallBack<WindowInstance>() {
+ @Override
+ public boolean flushMessage(List<WindowInstance> windowInstances) {
+
+ Collections.sort(windowInstances, new Comparator<WindowInstance>() {
+ @Override
+ public int compare(WindowInstance o1, WindowInstance o2) {
+ int value= o1.getFireTime().compareTo(o2.getFireTime());
+ if(value!=0){
+ return value;
+ }
+ return o2.getStartTime().compareTo(o1.getStartTime());
+ }
+ });
+
+ for(WindowInstance windowInstance:windowInstances){
+ fireWindowInstance(windowInstance);
+ }
+ return true;
+ }
+ });
+ }
+
+
+ @Override
+ protected String createSplitId(WindowInstance windowInstance) {
+ return windowInstance.getSplitId();
+ }
+ }
+
+ @Override
+ public boolean supportNewSplitFind() {
+ return true;
+ }
+
+ @Override
+ public boolean supportRemoveSplitFind() {
+ return false;
+ }
+
+ @Override
+ public boolean supportOffsetRest() {
+ return false;
+ }
+
+ @Override
+ protected boolean isNotDataSplit(String queueId) {
+ return false;
+ }
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/WindowBaseValue.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/WindowBaseValue.java
new file mode 100644
index 0000000..d558c20
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/WindowBaseValue.java
@@ -0,0 +1,188 @@
+/*
+ * 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.state;
+
+import java.io.Serializable;
+import java.util.Date;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
+import org.apache.rocketmq.streams.common.utils.DateUtil;
+import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.utils.ReflectUtil;
+
+public class WindowBaseValue extends BasedConfigurable implements Serializable {
+
+ private static final Log LOG = LogFactory.getLog(WindowBaseValue.class);
+
+ private static final long serialVersionUID = -4985883726971532986L;
+
+ /**
+ * 唯一标识一个groupBy的窗口计算数据 创建唯一主键 内存及磁盘里使用(partition,windowNamespace,windowName,startTime,endOrFireTime,groupByValue)的拼接 数据库里用上面拼接字符串的MD5
+ */
+ protected String msgKey;
+
+ /**
+ * 唯一标识一个窗口 内存及磁盘使用(patitionId,windowNamespace,windowName,startTime,endOrFireTime) 数据库里使用上面字符串的MD5
+ */
+ protected String windowInstanceId;
+
+ /**
+ * 分片信息(metaQ里是queue)
+ */
+ protected String partition;
+
+ /**
+ * 同一分片同一窗口的自增数据(增加逻辑在业务里,为什么不使用id?)
+ */
+ protected long partitionNum;
+
+ /**
+ * 标识一个分片同一个窗口 内存及磁盘使用(partition,windowNamespace,windowName,windowinstanceName,startTime,endTime,partition) 数据库里使用上面字符串的MD5值
+ */
+ protected String windowInstancePartitionId;
+
+ /**
+ * 窗口实例的开始时间
+ */
+ protected String startTime;
+
+ /**
+ * 窗口实例的结束时间
+ */
+ protected String endTime;
+
+ /**
+ * 窗口实例的触发时间
+ */
+ protected String fireTime;
+
+ protected AtomicLong updateVersion = new AtomicLong(0);
+
+ public WindowBaseValue() {
+ setGmtCreate(DateUtil.getCurrentTime());
+ setGmtModified(DateUtil.getCurrentTime());
+ }
+
+ public long incrementUpdateVersion() {
+ return updateVersion.incrementAndGet();
+ }
+
+ @Override
+ public Date getGmtCreate() {
+ return gmtCreate;
+ }
+
+ @Override
+ public void setGmtCreate(Date gmtCreate) {
+ this.gmtCreate = gmtCreate;
+ }
+
+ @Override
+ public Date getGmtModified() {
+ return gmtModified;
+ }
+
+ @Override
+ public void setGmtModified(Date gmtModified) {
+ this.gmtModified = gmtModified;
+ }
+
+ public String getMsgKey() {
+ return msgKey;
+ }
+
+ public void setMsgKey(String msgKey) {
+ this.msgKey = msgKey;
+ }
+
+ public String getWindowInstanceId() {
+ return windowInstanceId;
+ }
+
+ public void setWindowInstanceId(String windowInstanceId) {
+ this.windowInstanceId = windowInstanceId;
+ }
+
+ public String getPartition() {
+ return partition;
+ }
+
+ public void setPartition(String partition) {
+ this.partition = partition;
+ }
+
+ public long getPartitionNum() {
+ return partitionNum;
+ }
+
+ public void setPartitionNum(long partitionNum) {
+ this.partitionNum = partitionNum;
+ }
+
+ public String getStartTime() {
+ return startTime;
+ }
+
+ public void setStartTime(String startTime) {
+ this.startTime = startTime;
+ }
+
+ public String getEndTime() {
+ return endTime;
+ }
+
+ public void setEndTime(String endTime) {
+ this.endTime = endTime;
+ }
+
+ public String getFireTime() {
+ return fireTime;
+ }
+
+ public void setFireTime(String fireTime) {
+ this.fireTime = fireTime;
+ }
+
+ public long getUpdateVersion() {
+ return updateVersion.get();
+ }
+
+ public void setUpdateVersion(long updateVersion) {
+ this.updateVersion.set(updateVersion);
+ }
+
+ public String getWindowInstancePartitionId() {
+ return windowInstancePartitionId;
+ }
+
+ public void setWindowInstancePartitionId(String windowInstancePartitionId) {
+ this.windowInstancePartitionId = windowInstancePartitionId;
+ }
+
+ @Override
+ public WindowBaseValue clone() {
+ String json = this.toJson();
+ WindowBaseValue clonedValue = ReflectUtil.forInstance(this.getClass());
+ clonedValue.toObject(json);
+ return clonedValue;
+ }
+
+}
+
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/JoinLeftState.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/JoinLeftState.java
new file mode 100644
index 0000000..63dfe07
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/JoinLeftState.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.window.state.impl;
+
+public class JoinLeftState extends JoinState {
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/JoinRightState.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/JoinRightState.java
new file mode 100644
index 0000000..eb69161
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/JoinRightState.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.window.state.impl;
+
+public class JoinRightState extends JoinState {
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/JoinState.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/JoinState.java
new file mode 100644
index 0000000..3a85ecd
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/JoinState.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.window.state.impl;
+
+import java.util.Date;
+
+import org.apache.rocketmq.streams.window.state.WindowBaseValue;
+
+public class JoinState extends WindowBaseValue {
+
+ private long windowId;
+
+ private String windowName;
+
+ private String windowNameSpace;
+
+ private String messageId;
+
+ private String messageKey;
+
+ private Date messageTime;
+
+ private String messageBody;
+
+ public long getWindowId() {
+ return windowId;
+ }
+
+ public void setWindowId(long windowId) {
+ this.windowId = windowId;
+ }
+
+ public String getWindowName() {
+ return windowName;
+ }
+
+ public void setWindowName(String windowName) {
+ this.windowName = windowName;
+ }
+
+ public String getWindowNameSpace() {
+ return windowNameSpace;
+ }
+
+ public void setWindowNameSpace(String windowNameSpace) {
+ this.windowNameSpace = windowNameSpace;
+ }
+
+ public String getMessageId() {
+ return messageId;
+ }
+
+ public void setMessageId(String messageId) {
+ this.messageId = messageId;
+ }
+
+ public String getMessageKey() {
+ return messageKey;
+ }
+
+ public void setMessageKey(String messageKey) {
+ this.messageKey = messageKey;
+ }
+
+ public Date getMessageTime() {
+ return messageTime;
+ }
+
+ public void setMessageTime(Date messageTime) {
+ this.messageTime = messageTime;
+ }
+
+ public String getMessageBody() {
+ return messageBody;
+ }
+
+ public void setMessageBody(String messageBody) {
+ this.messageBody = messageBody;
+ }
+}
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
new file mode 100644
index 0000000..da6097d
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java
@@ -0,0 +1,548 @@
+/*
+ * 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.state.impl;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+import com.alibaba.fastjson.JSON;
+import com.alibaba.fastjson.JSONArray;
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.streams.common.interfaces.IStreamOperator;
+import org.apache.rocketmq.streams.common.utils.Base64Utils;
+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;
+import org.apache.rocketmq.streams.script.context.FunctionContext;
+import org.apache.rocketmq.streams.common.datatype.DataType;
+import org.apache.rocketmq.streams.common.datatype.NotSupportDataType;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.apache.rocketmq.streams.common.utils.DataTypeUtil;
+import org.apache.rocketmq.streams.common.utils.DateUtil;
+import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
+import org.apache.rocketmq.streams.common.utils.ReflectUtil;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+import org.apache.rocketmq.streams.common.utils.TraceUtil;
+import org.apache.rocketmq.streams.script.operator.impl.AggregationScript;
+import org.apache.rocketmq.streams.script.operator.impl.FunctionScript;
+import org.apache.rocketmq.streams.script.service.IAccumulator;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
+
+public class WindowValue extends WindowBaseValue implements Serializable {
+
+ private static final long serialVersionUID = 1083444850264401338L;
+
+ private static final Log LOG = LogFactory.getLog(WindowValue.class);
+
+ /**
+ * 如果做分组,设置分组
+ */
+ protected String groupBy = "0";
+
+ /**
+ * split id和max offset的映射关系
+ */
+ protected ConcurrentHashMap<String, String> maxOffset = new ConcurrentHashMap<>(16);
+
+ /**
+ * the result of aggregation column
+ */
+ private Map<String, Object> aggColumnResult = new ConcurrentHashMap<>(16);
+
+ /**
+ * the result of select column
+ */
+ protected Map<String, Object> computedColumnResult = new HashMap<>(16);
+
+ protected transient Long lastUpdateTime;//used in session window,set last update time
+
+ public WindowValue() {
+ setGmtCreate(DateUtil.getCurrentTime());
+ setGmtModified(DateUtil.getCurrentTime());
+ }
+
+ @Override
+ protected void getJsonObject(JSONObject jsonObject) {
+ super.getJsonObject(jsonObject);
+ String result = jsonObject.getString("aggColumnResult");
+ setAggColumnResult(result);
+ }
+
+ @Override
+ protected void setJsonObject(JSONObject jsonObject) {
+ super.setJsonObject(jsonObject);
+ if (aggColumnResult == null) {
+ return;
+ }
+ jsonObject.put("aggColumnResult", getAggColumnResult());
+
+ }
+
+ public WindowValue(WindowValue theValue) {
+ this.startTime = theValue.getStartTime();
+ this.endTime = theValue.getEndTime();
+ this.fireTime = theValue.getFireTime();
+ this.groupBy = theValue.getGroupBy();
+ setNameSpace(theValue.getNameSpace());
+ setConfigureName(theValue.getConfigureName());
+ }
+
+ /**
+ * 计算结果序列化成json
+ *
+ * @return
+ */
+ public String getAggColumnResult() {
+ JSONArray jsonArray = new JSONArray();
+
+ Iterator<Entry<String, Object>> it = aggColumnResult.entrySet().iterator();
+ while (it.hasNext()) {
+ JSONObject jsonObject = new JSONObject();
+ Entry<String, Object> entry = it.next();
+ String functionName = entry.getKey();
+ Object value = entry.getValue();
+ jsonObject.put("function", functionName);
+ if (value == null) {
+ continue;
+ }
+ DataType dataType = DataTypeUtil.getDataTypeFromClass(value.getClass());
+ boolean isBasicType = false;
+ String jsonValue = null;
+ if (!NotSupportDataType.class.isInstance(dataType)) {
+ isBasicType = true;
+ jsonValue = dataType.toDataJson(value);
+ jsonObject.put("datatype", dataType.getDataTypeName());
+ } else {
+ isBasicType = false;
+ jsonValue = ReflectUtil.serializeObject(value).toJSONString();
+ }
+
+ jsonObject.put("isBasic", isBasicType);
+ jsonObject.put("result", jsonValue);
+ jsonArray.add(jsonObject);
+ }
+ return encodeSQLContent(jsonArray.toJSONString());
+ }
+
+ /**
+ * 还原计算结果
+ *
+ * @param jsonArrayStr
+ */
+ public void setAggColumnResult(String jsonArrayStr) {
+ jsonArrayStr = decodeSQLContent(jsonArrayStr);
+ JSONArray functionResultJson = JSONArray.parseArray(jsonArrayStr);
+ for (int i = 0; i < functionResultJson.size(); i++) {
+ JSONObject jsonObject = functionResultJson.getJSONObject(i);
+ String functionName = jsonObject.getString("function");
+ Boolean isBasic = jsonObject.getBoolean("isBasic");
+ String jsonValue = jsonObject.getString("result");
+ Object value = null;
+ if (isBasic) {
+ String datatype = jsonObject.getString("datatype");
+ DataType dataType = DataTypeUtil.getDataType(datatype);
+ value = dataType.getData(jsonValue);
+ } else {
+ JSONObject objectJson = JSONObject.parseObject(jsonValue);
+ value = ReflectUtil.deserializeObject(objectJson);
+ }
+ aggColumnResult.put(functionName, value);
+ }
+ }
+
+ public void setAggColumnMap(Map<String, Object> aggColumnResult) {
+ this.aggColumnResult = aggColumnResult;
+ }
+
+ public void setComputedColumnResult(String computedColumnResult) {
+ computedColumnResult = decodeSQLContent(computedColumnResult);
+ this.computedColumnResult = Message.parseObject(computedColumnResult);
+ }
+
+ public String getComputedColumnResult() {
+ JSONObject object = null;
+ if (JSONObject.class.isInstance(computedColumnResult)) {
+ object = (JSONObject)computedColumnResult;
+ } else {
+ object = new JSONObject(computedColumnResult);
+ }
+ return encodeSQLContent(object.toJSONString());
+ }
+
+ public void setMaxOffset(String theOffset) {
+ JSONObject object = JSONObject.parseObject(theOffset);
+ Iterator<String> iterator = object.keySet().iterator();
+ while (iterator.hasNext()) {
+ String key = iterator.next();
+ maxOffset.put(key, object.getString(key));
+ }
+ }
+
+ public String getMaxOffset() {
+ JSONObject object = new JSONObject();
+ object.putAll(maxOffset);
+ return object.toJSONString();
+ }
+
+ public Iterator<Entry<String, Object>> iteratorComputedColumnResult() {
+ return computedColumnResult.entrySet().iterator();
+ }
+
+ public Object getComputedColumnResultByKey(String fieldName) {
+ return computedColumnResult.get(fieldName);
+ }
+
+ public Map<String, Object> getcomputedResult() {
+ return this.computedColumnResult;
+ }
+
+ /**
+ * @param window all kinds of configurable information
+ * @param message the consumed data
+ */
+ public boolean calculate(AbstractWindow window, IMessage message) {
+ message.getMessageBody().put(AbstractWindow.WINDOW_START, startTime);
+ message.getMessageBody().put(AbstractWindow.WINDOW_END, endTime);
+ //每个计算节点对应一个consume split,如果多于一个queue的话,
+ String queueId = message.getHeader().getQueueId();
+ String offset = message.getHeader().getOffset();
+ if (StringUtil.isEmpty(offset)) {
+ offset = String.valueOf(System.currentTimeMillis());
+ }
+ String maxOffsetOfQueue = this.maxOffset.get(queueId);
+ if (StringUtil.isEmpty(maxOffsetOfQueue)) {
+ maxOffsetOfQueue = offset;
+ this.maxOffset.put(queueId, maxOffsetOfQueue);
+ } else {
+ if (message.getHeader().greateThan(maxOffsetOfQueue)) {
+ this.maxOffset.put(queueId, offset);
+ } else {
+ //如果比最大的offset 小或等于,则直接丢弃掉消息
+ System.out.println("!!!!!!!!!!!!!!!!!!! has outOfOrder data");
+ return false;
+ }
+ }
+ try {
+ this.lastUpdateTime = WindowInstance.getOccurTime(window, message);
+ if (window.getReduceSerializeValue() != null) {
+ JSONObject accumulator = null;
+ if (computedColumnResult != null && JSONObject.class.isInstance(computedColumnResult)) {
+ accumulator = (JSONObject)computedColumnResult;
+ }
+
+ JSONObject result = window.getReducer().reduce(accumulator, message.getMessageBody());
+ computedColumnResult = result;
+ return true;
+ }
+ 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", getComputedColumnResult());
+ }
+ } catch (Exception e) {
+ LOG.error("failed in calculating the message", e);
+ }
+
+ //there is no need writing back to message
+
+ return true;
+ }
+
+ protected void calFunctionColumn(AbstractWindow window, IMessage message) {
+ for (Entry<String, List<FunctionExecutor>> entry : window.getColumnExecuteMap().entrySet()) {
+ String computedColumn = entry.getKey();
+ List<FunctionExecutor> fifoQueue = entry.getValue();
+ for (FunctionExecutor operator : fifoQueue) {
+ String executorName = operator.getColumn();
+ IStreamOperator<IMessage, List<IMessage>> executor = operator.getExecutor();
+ if (executor instanceof AggregationScript) {
+ AggregationScript originAccScript = (AggregationScript)executor;
+ AggregationScript windowAccScript = originAccScript.clone();
+ Object accumulator = null;
+ if (aggColumnResult.containsKey(executorName)) {
+ accumulator = aggColumnResult.get(executorName);
+ } else {
+ IAccumulator director = AggregationScript.getAggregationFunction(
+ windowAccScript.getFunctionName());
+ accumulator = director.createAccumulator();
+ aggColumnResult.put(executorName, accumulator);
+ }
+ windowAccScript.setAccumulator(accumulator);
+ message.getMessageBody().put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY,
+ AggregationScript.INNER_AGGREGATION_COMPUTE_SINGLE);
+ FunctionContext context = new FunctionContext(message);
+ windowAccScript.doMessage(message, context);
+ } else if (executor instanceof FunctionScript) {
+ FunctionContext context = new FunctionContext(message);
+ ((FunctionScript)executor).doMessage(message, context);
+ }
+ }
+ //
+ computedColumnResult.put(computedColumn, message.getMessageBody().get(computedColumn));
+ }
+ }
+
+ protected void calProjectColumn(AbstractWindow window, IMessage message) {
+ Map<String, String> constMap = window.getColumnProjectMap();
+ for (Entry<String, String> entry : constMap.entrySet()) {
+ String computedColumn = entry.getKey();
+ String originColumn = entry.getValue();
+ if (message.getMessageBody().containsKey(originColumn)) {
+ computedColumnResult.put(computedColumn, message.getMessageBody().get(originColumn));
+ } else {
+ if (LOG.isWarnEnabled()) {
+ LOG.warn("field:\t " + originColumn + " lost!");
+ }
+ }
+ }
+ }
+
+ /**
+ * merge different window values into one window value which have the same group by value
+ *
+ * @param window the window definition
+ * @param windowInstances all window instance which belong to same window and have different group by value
+ * @return
+ */
+ public static List<WindowValue> mergeWindowValues(AbstractWindow window, List<WindowInstance> windowInstances) {
+ if (windowInstances == null || windowInstances.size() == 0) {
+ return new ArrayList<>();
+ }
+ StringBuilder sb = new StringBuilder();
+ boolean isFirst = true;
+ String name = MapKeyUtil.createKey(window.getNameSpace(), window.getConfigureName());
+ for (WindowInstance windowInstance : windowInstances) {
+ if (isFirst) {
+ isFirst = false;
+ } else {
+ sb.append(",");
+ }
+ sb.append("('" + name + "','" + windowInstance.getStartTime() + "','" + windowInstance.getEndTime() + "')");
+ }
+ String inSQL = sb.toString();
+ /**
+ * 分批,内存撑暴 todo
+ */
+ String sql = "select * from " + ORMUtil
+ .getTableName(WindowValue.class) + " where status > 0 && (name, start_time, end_time) in (" + inSQL + ")";
+ Map<String, Object> paras = new HashMap<>(4);
+ List<WindowValue> windowValueList = ORMUtil.queryForList(sql, paras, WindowValue.class);
+ return queryMergeWindowValues(window, windowValueList);
+ }
+
+ public static List<WindowValue> queryMergeWindowValues(AbstractWindow window, List<WindowValue> windowValueList) {
+ Map<String, List<WindowValue>> groupWindowMap = new HashMap<>(64);
+ for (WindowValue value : windowValueList) {
+ String key = MapKeyUtil.createKeyBySign(value.getStartTime(), value.getEndTime(),
+ value.getGroupBy());
+ if (groupWindowMap.containsKey(key)) {
+ groupWindowMap.get(key).add(value);
+ } else {
+ groupWindowMap.put(key, new ArrayList<WindowValue>() {{
+ add(value);
+ }});
+ }
+ }
+ List<WindowValue> mergedValueList = new ArrayList<>();
+ for (Entry<String, List<WindowValue>> entry : groupWindowMap.entrySet()) {
+ mergedValueList.add(mergeWindowValue(window, entry.getValue()));
+ }
+ return mergedValueList;
+ }
+
+ /**
+ * merge the group which has the same group by value and different split id
+ */
+ private static WindowValue mergeWindowValue(AbstractWindow window, List<WindowValue> valueList) {
+ WindowValue lastWindowValue = new WindowValue(valueList.get(0));
+ lastWindowValue.computedColumnResult = (Map<String, Object>)JSON.parse(
+ valueList.get(0).getComputedColumnResult());
+ //
+ for (Entry<String, List<FunctionExecutor>> entry : window.getColumnExecuteMap().entrySet()) {
+ String computedColumn = entry.getKey();
+ IMessage message = new Message(new JSONObject());
+ FunctionContext context = new FunctionContext(message);
+ List<FunctionExecutor> executorList = entry.getValue();
+ //column outside of the aggregation function should be calculated again!
+ boolean needMergeComputation = false;
+ for (FunctionExecutor info : executorList) {
+ String column = info.getColumn();
+ IStreamOperator<IMessage, List<IMessage>> engine = info.getExecutor();
+ if (engine instanceof AggregationScript) {
+ AggregationScript origin = (AggregationScript)engine;
+ AggregationScript operator = origin.clone();
+ if (needMergeComputation) {
+ message.getMessageBody().put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY,
+ AggregationScript.INNER_AGGREGATION_COMPUTE_SINGLE);
+ operator.setAccumulator(operator.getDirector().createAccumulator());
+ operator.doMessage(message, context);
+ } else {
+ message.getMessageBody().put(AggregationScript.INNER_AGGREGATION_COMPUTE_KEY,
+ AggregationScript.INNER_AGGREGATION_COMPUTE_MULTI);
+ List actors = valueList.stream().map(
+ windowValue -> windowValue.getAccumulatorByColumn(column)).collect(
+ Collectors.toList());
+ operator.setAccumulator(operator.getDirector().createAccumulator());
+ operator.setAccumulators(actors);
+ operator.doMessage(message, context);
+ needMergeComputation = true;
+ }
+ } else if (engine instanceof FunctionScript) {
+ FunctionScript theScript = (FunctionScript)engine;
+ String[] parameters = theScript.getDependentParameters();
+ for (String parameter : parameters) {
+ if (!message.getMessageBody().containsKey(parameter) && lastWindowValue.computedColumnResult
+ .containsKey(parameter)) {
+ message.getMessageBody().put(parameter,
+ lastWindowValue.computedColumnResult.get(parameter));
+ }
+ }
+ if (needMergeComputation) {
+ engine.doMessage(message, context);
+ }
+ }
+ }
+ if (message.getMessageBody().containsKey(computedColumn)) {
+ lastWindowValue.computedColumnResult.put(computedColumn, message.getMessageBody().get(computedColumn));
+ } else if (!needMergeComputation) {
+ lastWindowValue.computedColumnResult.put(computedColumn,
+ valueList.get(0).computedColumnResult.get(computedColumn));
+ }
+ }
+ // valueList.stream().map(value -> lastWindowValue.count += value.getCount());
+ //
+ List<String> traceList = new ArrayList<>();
+ for (WindowValue value : valueList) {
+ if (value.computedColumnResult.containsKey(TraceUtil.TRACE_ID_FLAG)) {
+ String traceIds = String.valueOf(value.computedColumnResult.get(TraceUtil.TRACE_ID_FLAG));
+ traceList.addAll(Arrays.asList(traceIds.split(",")));
+ }
+ }
+ if (!traceList.isEmpty()) {
+ StringBuffer buffer = new StringBuffer();
+ for (int i = 0; i < traceList.size(); i++) {
+ buffer.append(traceList.get(i));
+ if (i != traceList.size() - 1) {
+ buffer.append(",");
+ }
+ }
+ lastWindowValue.computedColumnResult.put(TraceUtil.TRACE_ID_FLAG, buffer.toString());
+ }
+ return lastWindowValue;
+ }
+
+ public Object getAccumulatorByColumn(String column) {
+ return aggColumnResult.getOrDefault(column, null);
+ }
+
+ public String getGroupBy() {
+ return groupBy;
+ }
+
+ public void setGroupBy(String groupBy) {
+ this.groupBy = groupBy;
+ }
+
+ @Override
+ public WindowValue clone() {
+ WindowValue clonedValue = new WindowValue();
+ clonedValue.setId(getId());
+ clonedValue.setGmtModified(gmtModified);
+ clonedValue.setGmtCreate(gmtCreate);
+ clonedValue.setEndTime(endTime);
+ clonedValue.setStartTime(startTime);
+ clonedValue.setFireTime(fireTime);
+ clonedValue.setConfigureName(getConfigureName());
+ clonedValue.setNameSpace(getNameSpace());
+ clonedValue.setMsgKey(msgKey);
+ clonedValue.setAggColumnMap(aggColumnResult);
+ clonedValue.setMaxOffset(getMaxOffset());
+ clonedValue.setWindowInstancePartitionId(windowInstancePartitionId);
+ clonedValue.setWindowInstanceId(windowInstanceId);
+ clonedValue.setPartition(partition);
+ clonedValue.setPartitionNum(partitionNum);
+ clonedValue.setGroupBy(groupBy);
+ clonedValue.setAggColumnResult(getAggColumnResult());
+ clonedValue.setComputedColumnResult(getComputedColumnResult());
+ clonedValue.setUpdateVersion(getUpdateVersion());
+ clonedValue.setVersion(getVersion());
+ clonedValue.setUpdateFlag(getUpdateFlag());
+ 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 WindowValue toOriginValue(boolean supportOutDate) {
+ // WindowValue clonedValue = clone();
+ // String windowInstanceId = WindowInstance.getWindowInstanceId(getNameSpace(), getConfigureName(), getStartTime(),
+ // getEndTime(), getFireTime(), supportOutDate);
+ // clonedValue.setMsgKey(MapKeyUtil
+ // .createKey(getPartition(), windowInstanceId, getGroupBy()));
+ // clonedValue.setWindowInstanceId(windowInstanceId);
+ // clonedValue.setWindowInstancePartitionId(
+ // MapKeyUtil.createKey(windowInstanceId, getPartition()));
+ // return clonedValue;
+ //}
+
+ public Long getLastUpdateTime() {
+ return lastUpdateTime;
+ }
+
+ public void setLastUpdateTime(Long lastUpdateTime) {
+ this.lastUpdateTime = lastUpdateTime;
+ }
+
+ protected String encodeSQLContent(String content) {
+ try {
+ return Base64Utils.encode(content.getBytes("UTF-8"));
+ } catch (Exception e) {
+ throw new RuntimeException("encode sql content error " + content, e);
+ }
+ }
+
+ protected String decodeSQLContent(String sqlContent) {
+ try {
+ return new String(Base64Utils.decode(sqlContent), "UTF-8");
+ } catch (Exception e) {
+ throw new RuntimeException("decode sql content error " + sqlContent, e);
+ }
+ }
+}
+
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractWindowStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractWindowStorage.java
new file mode 100644
index 0000000..80514cb
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractWindowStorage.java
@@ -0,0 +1,120 @@
+/*
+ * 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.storage;
+
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+import org.apache.rocketmq.streams.db.driver.batchloader.BatchRowLoader;
+import org.apache.rocketmq.streams.db.driver.batchloader.IRowOperator;
+import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
+import org.apache.rocketmq.streams.window.state.WindowBaseValue;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+public abstract class AbstractWindowStorage<T extends WindowBaseValue> implements IWindowStorage<T> {
+ protected boolean isLocalStorageOnly = false;
+ protected transient ExecutorService dataLoaderExecutor = new ThreadPoolExecutor(10, 10,
+ 0L, TimeUnit.MILLISECONDS,
+ new LinkedBlockingQueue<Runnable>());
+ ;
+
+ @Override
+ public Long getMaxShuffleId(String queueId, String windowNameSpace, String windowName, Class<T> clazz) {
+ if (isLocalStorageOnly) {
+ return null;
+ }
+ String sql = "select max(partition_num) as partition_num from " + ORMUtil.getTableName(clazz)
+ + " where name_space ='" + windowNameSpace + "' and configure_name='" + windowName + "' and `partition`='" + queueId + "'";
+ WindowBaseValue windowBaseValue = ORMUtil.queryForObject(sql, new HashMap<>(4), clazz);
+ if (windowBaseValue == null) {
+ return null;
+ }
+ return windowBaseValue.getPartitionNum();
+ }
+
+ @Override
+ public void multiPut(Map<String, T> map, String windowInstanceId, String queueId) {
+ throw new RuntimeException("can not support this method");
+ }
+
+ @Override
+ public Map<String, T> multiGet(Class<T> clazz, List<String> keys, String windowInstanceId, String queueId) {
+ throw new RuntimeException("can not support this method");
+ }
+
+ @Override
+ public void loadSplitData2Local(String queueId, String windowInstanceId, Class<T> clazz, IRowOperator processor) {
+ if (isLocalStorageOnly) {
+ return;
+ }
+ String windowInstancePartitionId = StringUtil.createMD5Str(windowInstanceId);
+ dataLoaderExecutor.execute(new Runnable() {
+ @Override
+ public void run() {
+ long start = System.currentTimeMillis();
+ BatchRowLoader batchRowLoader = new BatchRowLoader("partition_num",
+ "select * from " + ORMUtil.getTableName(clazz) + " where window_instance_partition_id ='"
+ + windowInstancePartitionId + "'", processor);
+ batchRowLoader.startLoadData();
+ ShufflePartitionManager.getInstance().setWindowInstanceFinished(windowInstanceId);
+ System.out.println(System.currentTimeMillis() - start);
+ System.out.println("");
+ }
+ });
+
+ }
+
+ @Override
+ public void put(String key, T value) {
+ Map<String, T> map = new HashMap<>();
+ map.put(key, value);
+ multiPut(map);
+ }
+
+ @Override
+ public T get(Class<T> clazz, String key) {
+ Map<String, T> result = multiGet(clazz, key);
+ if (result == null) {
+ return null;
+ }
+ return result.values().iterator().next();
+ }
+
+ @Override
+ public Map<String, T> multiGet(Class<T> clazz, String... keys) {
+ List<String> keyList = new ArrayList<>();
+ for (String key : keys) {
+ keyList.add(key);
+ }
+ return multiGet(clazz, keyList);
+ }
+
+ public boolean isLocalStorageOnly() {
+ return isLocalStorageOnly;
+ }
+
+ public void setLocalStorageOnly(boolean localStorageOnly) {
+ isLocalStorageOnly = localStorageOnly;
+ }
+
+}
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
new file mode 100644
index 0000000..95003ac
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ICommonStorage.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.window.storage;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * 对存储的统一抽象,最底层的抽象 T必须是可识别的对象
+ */
+public interface ICommonStorage<T> {
+
+ /**
+ * 支持单个key value的存储
+ *
+ * @param key
+ * @param value
+ */
+ void put(String key, T value);
+
+ //多组key value批量存储
+ void multiPut(Map<String, T> map);
+
+ //获取多个key的值
+ Map<String, T> multiGet(Class<T> clazz, List<String> keys);
+
+ //获取单个key的值
+ T get(Class<T> clazz, String key);
+
+ //获取多个key的值
+ Map<String, T> multiGet(Class<T> clazz, String... keys);
+
+ void removeKeys(Collection<String> keys);
+
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IKeyGenerator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IKeyGenerator.java
new file mode 100644
index 0000000..cb01eff
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IKeyGenerator.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.window.storage;
+
+/**
+ * 为了适配不同存储,可以通过key generator的实现来做非kv系统的映射
+ */
+public interface IKeyGenerator {
+
+ /**
+ * @param
+ * @return
+ */
+ String createKey();
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IShufflePartitionManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IShufflePartitionManager.java
new file mode 100644
index 0000000..6fd4639
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IShufflePartitionManager.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.window.storage;
+
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+
+import java.util.Collection;
+
+public interface IShufflePartitionManager {
+
+ /**
+ * 这个分片是否可用本地存储
+ *
+ * @param shuffleId
+ * @return
+ */
+ boolean isLocalStorage(String shuffleId, String windowInstanceId);
+
+ void setLocalStorageInvalid(ISplit channelQueue);
+
+ void setLocalStorageInvalid(ISplit channelQueue, String windowInstanceId);
+
+ /**
+ * setLocalStorageInvalid 如果 shuffle id不存在,且
+ *
+ * @param shuffleId
+ */
+ boolean setLocalStorageValdateIfNotExist(String shuffleId, String windowInstanceId);
+
+ /**
+ * 当窗口实例触发后,通过这个方法,回收资源
+ *
+ * @param windowInstanceId
+ * @param queueIds
+ */
+ void clearWindowInstanceStorageStatus(String windowInstanceId, Collection<String> queueIds);
+
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IStorage.java
new file mode 100644
index 0000000..de7e566
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IStorage.java
@@ -0,0 +1,79 @@
+/*
+ * 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.storage;
+
+import org.apache.rocketmq.streams.db.driver.batchloader.IRowOperator;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+public interface IStorage<T> {
+
+
+ /**
+ * 支持单个key value的存储
+ * @param key
+ * @param value
+ */
+ void put(String key,T value);
+
+ //多组key value批量存储
+ void mutilPut(Map<String,T> keyValue);
+
+ //获取单个key的值
+ T get(Class<T> clazz,IKeyGenerator keyGenerator,String key);
+
+ //获取多个key的值
+ Map<String,T> mutilGet(Class<T> clazz,IKeyGenerator keyGenerator,String... keyValues);
+ //获取多个key的值
+ Map<String,T> mutilGet(Class< T> clazz,IKeyGenerator keyGenerator, List<String> keys);
+
+ /**
+ * remove keys
+ * @param keys
+ */
+ void removeKeys(IKeyGenerator keyGenerator, Collection<String> keys);
+
+ /**
+ * remove keys by prefix
+ * @param keyPrefix
+ */
+ void removeKeyPrefix(IKeyGenerator keyGenerator,String keyPrefix);
+
+ /*
+ create Iterator by key prefix
+ */
+ Iterator<T> iterateByPrefix(IKeyGenerator keyGenerator,String keyPrefix,Class<T> clazz);
+
+
+ T putIfAbsent(T t,Class<T> clazz);
+
+
+ int count(IKeyGenerator keyGenerator,String key);
+
+ int incrementAndGet(IKeyGenerator keyGenerator,String key);
+
+
+ Iterator<T> queryByPrefixBetweenOrderByValue(IKeyGenerator keyGenerator,String keyPrefix, Object startIndexValue,Object endIndexValue,Class<T> clazz);
+
+
+
+ void loadByPrefixBetweenOrderByValue(IKeyGenerator keyGenerator,String keyPrefix, Object startIndexValue,Object endIndexValue,
+ IRowOperator rowOperator,Class<T> clazz);
+
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IWindowStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IWindowStorage.java
new file mode 100644
index 0000000..4c55cd7
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IWindowStorage.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.streams.window.storage;
+
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.db.driver.batchloader.IRowOperator;
+import org.apache.rocketmq.streams.window.state.WindowBaseValue;
+import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public interface IWindowStorage<T extends WindowBaseValue> extends ICommonStorage<T> {
+
+ //多组key value批量存储
+ void multiPut(Map<String, T> map, String windowInstanceId, String queueId);
+
+ //获取多个key的值
+ Map<String, T> multiGet(Class<T> clazz, List<String> keys, String windowInstanceId, String queueId);
+
+ /***
+ * 把queueId 前缀的数据全部失效掉
+ * @param channelQueue 必须6位,64001 1280001 128分片总数,001第一个分片
+ */
+
+ void clearCache(ISplit channelQueue, Class<T> clazz);
+
+ /**
+ * 删除一个窗口实例的数据,包括远程和本地存储
+ */
+ void delete(String windowInstanceId, Set<String> queueIds, Class<T> clazz);
+
+ /**
+ * 加载一个窗口实例的数据到本地存储
+ */
+ WindowBaseValueIterator<T> loadWindowInstanceSplitData(String localStorePrefix, String queueId, String windowInstanceId, String keyPrefix,
+ Class<T> clazz);
+
+ /**
+ * 这个窗口实例,最大的分片序列号,主要是通过db获取
+ *
+ * @return
+ */
+ Long getMaxShuffleId(String queueId, String windowNameSpace, String windowName, Class<T> clazz);
+
+ /**
+ * 批量加载数据,放入本地缓存
+ *
+ * @param splitNumer
+ * @param rowOperator
+ */
+ void loadSplitData2Local(String splitNumer, String windowInstanceId, Class<T> clazz, IRowOperator rowOperator);
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ShufflePartitionManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ShufflePartitionManager.java
new file mode 100644
index 0000000..9bc48a6
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ShufflePartitionManager.java
@@ -0,0 +1,86 @@
+/*
+ * 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.storage;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+public class ShufflePartitionManager {
+ private static ShufflePartitionManager instance = new ShufflePartitionManager();
+ protected Map<String, Boolean> splitId2AllWindowInstanceFinishInit = new HashMap<>();//split是否有效,这个分片下所有的window instacne都完成了初始化
+ protected Map<String, Boolean> windowInstanceId2FinishInit = new HashMap<>();//window instance 是否完成初始化
+ private ExecutorService executorService;
+
+ private ShufflePartitionManager() {
+ executorService = new ThreadPoolExecutor(10, 10,
+ 0L, TimeUnit.MILLISECONDS,
+ new LinkedBlockingQueue<Runnable>());
+
+ }
+
+ public static ShufflePartitionManager getInstance() {
+ return instance;
+ }
+
+ /**
+ * if window instance finished init, return true else return false
+ *
+ * @param splitId
+ * @param windowInstanceId
+ * @return
+ */
+ public boolean isWindowInstanceFinishInit(String splitId, String windowInstanceId) {
+ Boolean allSpliltFinish = splitId2AllWindowInstanceFinishInit.get(splitId);
+ if (allSpliltFinish != null && allSpliltFinish) {
+ return true;
+ }
+ Boolean windowInstanceInitFinished = windowInstanceId2FinishInit.get(windowInstanceId);
+ if (windowInstanceInitFinished != null && windowInstanceInitFinished) {
+ return true;
+ }
+ return false;
+ }
+
+ public synchronized void setSplitFinished(String splitId) {
+ splitId2AllWindowInstanceFinishInit.put(splitId, true);
+ }
+
+ public synchronized void setSplitInValidate(String splitId) {
+ splitId2AllWindowInstanceFinishInit.put(splitId, false);
+ }
+
+ public synchronized void setWindowInstanceFinished(String windowInstanceId) {
+ windowInstanceId2FinishInit.put(windowInstanceId, true);
+ }
+
+ public synchronized void clearWindowInstance(String windowInstanceId) {
+ windowInstanceId2FinishInit.remove(windowInstanceId);
+ }
+
+ public synchronized void clearSplit(String queueId) {
+ splitId2AllWindowInstanceFinishInit.remove(queueId);
+ for (String windowInstanceId : this.windowInstanceId2FinishInit.keySet()) {
+ if (windowInstanceId.startsWith(queueId)) {
+ this.windowInstanceId2FinishInit.remove(windowInstanceId);
+ }
+ }
+ }
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/StorageManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/StorageManager.java
new file mode 100644
index 0000000..a78c848
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/StorageManager.java
@@ -0,0 +1,307 @@
+/*
+ * 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.storage;
+
+/**
+ * 存储管理,根据分片本地存储是否有效,来选择对应的存储完成计算
+ */
+public class StorageManager {
+ //private ExecutorService executorService= Executors.newFixedThreadPool(10);
+ //
+ //private ExecutorService dbService = Executors.newSingleThreadExecutor();
+ //
+ //private static StorageManager storageManager=new StorageManager();
+ //private StorageManager(){}
+ //protected static IStorage rocksDBStorage=new RocksdbStorage();//本地存储
+ //protected static DBStorage dbStorage=new DBStorage();//jdbc 存储
+ ////0/null:false;1:true;2加载中
+ ////
+ //protected transient boolean isLocalOnly=false;//只支持本地存储
+ //protected ConcurrentHashMap<String,Integer> shuffleIdAndWindowInstance2IsLocal=new ConcurrentHashMap<>();//某个分片是否本地存储有效
+ //public static StorageManager getStorageManager(){
+ // return storageManager;
+ //}
+ //
+ //public static IShufflePartitionManager getShufflePartitionManager(){
+ // return storageManager;
+ //}
+ //
+ //public static IStorage getLocalStorage(){
+ // return rocksDBStorage;
+ //}
+ //
+ //public static IStorage getRemoteStorage(){
+ // return dbStorage;
+ //}
+ //
+ //@Override
+ //public void put(Map<String, WindowBaseValue> values, boolean onlyLocal) {
+ // if(onlyLocal){
+ // rocksDBStorage.put(values, true);
+ // return;
+ // }
+ // Map<String,WindowBaseValue> notLocalWindowBaseValues=new HashMap<>();
+ // Map<String,WindowBaseValue> localWindowBaseValues=new HashMap<>();
+ // Iterator<Entry<String, WindowBaseValue>> it = values.entrySet().iterator();
+ // while (it.hasNext()){
+ // Entry<String, WindowBaseValue>entry=it.next();
+ // boolean isLocal=isLocalStorage(entry.getValue().getPartition(),entry.getValue().getWindowInstanceId());
+ // if(isLocal){
+ // localWindowBaseValues.put(entry.getKey(),entry.getValue());
+ // }else {
+ // notLocalWindowBaseValues.put(entry.getKey(),entry.getValue());
+ // }
+ // }
+ // rocksDBStorage.put(values, false);
+ // if(isLocalOnly){
+ // return;
+ // }
+ // /**
+ // *
+ // */
+ // if (!CollectionUtil.isEmpty(localWindowBaseValues)) {
+ // //如果本地可用,可以用异步的方式写,提高写性能
+ // dbService.execute(new Runnable() {
+ // @Override
+ // public void run() {
+ // dbStorage.put(getMd5Value(localWindowBaseValues), false);
+ // }
+ // });
+ // } else if (!CollectionUtil.isEmpty(notLocalWindowBaseValues)) {
+ // //如果本地不可用,必须同步写
+ // dbStorage.put(getMd5Value(notLocalWindowBaseValues), false);
+ // }
+ //}
+ //
+ ///**
+ // * DB存储时用MD5,TODO 考虑使用aop
+ // */
+ //private Map<String, WindowBaseValue> getMd5Value(Map<String, WindowBaseValue> originMap) {
+ // Map<String, WindowBaseValue> valueMap = new HashMap<>(originMap.size());
+ // Iterator<Entry<String, WindowBaseValue>> iterator = originMap.entrySet().iterator();
+ // while (iterator.hasNext()) {
+ // Entry<String, WindowBaseValue> entry = iterator.next();
+ // WindowBaseValue value = entry.getValue();
+ // if (value instanceof WindowValue) {
+ // WindowValue md5Value = (WindowValue)entry.getValue();
+ // valueMap.put(entry.getKey(), md5Value.toMd5Value());
+ // } else {
+ // //TODO join的MD5计算逻辑
+ // valueMap.put(entry.getKey(), entry.getValue());
+ // }
+ // }
+ // return valueMap;
+ //}
+ //
+ //@Override
+ //public Map<String, WindowBaseValue> get(Collection<String> keys, Class<? extends WindowBaseValue> clazz) {
+ // Map<String, WindowBaseValue> result=new HashMap<>();
+ // if(isLocalOnly){
+ // result.putAll(rocksDBStorage.get(keys,clazz));
+ // return result;
+ // }
+ //
+ // List<String> notLocalKeys=new ArrayList<>();
+ // List<String> localKeys=new ArrayList<>();
+ // for(String key:keys){
+ // String[] values=MapKeyUtil.spliteKey(key);
+ // String shuffleId=values[0];
+ // boolean isLocal = isLocalStorage(shuffleId, WindowInstance.createWindowInstanceId(key));
+ // if(isLocal){
+ // localKeys.add(key);
+ // }else {
+ // notLocalKeys.add(key);
+ // }
+ // }
+ //
+ // result.putAll(rocksDBStorage.get(localKeys,clazz));
+ // result.putAll(dbStorage.get(notLocalKeys,clazz));
+ // return result;
+ //}
+ //
+ //@Override
+ //public void delete(String windowNameSpace, String windowName, String startTime, String endOrFireTime,
+ // Class<? extends WindowBaseValue> clazz) {
+ // executorService.execute(new Runnable() {
+ //
+ // @Override
+ // public void run() {
+ // rocksDBStorage.delete(windowNameSpace, windowName, startTime, endOrFireTime, clazz);
+ // if(!isLocalOnly){
+ // dbStorage.delete(windowNameSpace, windowName, startTime, endOrFireTime, clazz);
+ // }
+ //
+ // }
+ // });
+ //
+ //}
+ //
+ //@Override
+ //public void clearCache(ISplit channelQueue) {
+ // rocksDBStorage.clearCache(channelQueue);
+ //}
+ //
+ //@Override
+ //public Iterator<WindowBaseValue> loadWindowInstanceSplitData(String queueId, String windowNameSpace,
+ // String windowName, String startTime, String endOrFireTime, String key, Class<? extends WindowBaseValue> clazz) {
+ // boolean isLocal = isLocalStorage(queueId,
+ // WindowInstance.getWindowInstanceId(windowNameSpace, windowName, startTime, endOrFireTime));
+ // if(isLocal){
+ // return rocksDBStorage.loadWindowInstanceSplitData(queueId,windowNameSpace,windowName,startTime,endOrFireTime,key, clazz);
+ // }else {
+ // return dbStorage.loadWindowInstanceSplitData(queueId,windowNameSpace,windowName,startTime,endOrFireTime,key, clazz);
+ // }
+ //}
+ //
+ //@Override
+ //public long getMaxShuffleId(String queueId, String windowNameSpace, String windowName, String startTime,
+ // String endOrFireTime, Class<? extends WindowBaseValue> clazz) {
+ // if(isLocalOnly){
+ // return 0;
+ // }
+ // return dbStorage.getMaxShuffleId(queueId,windowNameSpace,windowName,startTime,endOrFireTime,clazz);
+ //}
+ //
+ //@Override
+ //public void loadSplitData2Local(String splitNumer, String windowNameSpace, String windowName,
+ // String startTime, String endOrFireTime, Class<? extends WindowBaseValue> clazz, IRowOperator rowOperator) {
+ // if(isLocalOnly){
+ // return;
+ // }
+ // String windowInstanceId = WindowInstance.getWindowInstanceId(windowNameSpace, windowName, startTime, endOrFireTime);
+ // Integer value=this.shuffleIdAndWindowInstance2IsLocal.get(MapKeyUtil.createKey(splitNumer,windowInstanceId));
+ // if(value!=null&&value!=0){
+ // return;
+ // }
+ // synchronized (this){
+ // value=this.shuffleIdAndWindowInstance2IsLocal.get(MapKeyUtil.createKey(splitNumer,windowInstanceId));
+ // if(value!=null&&value!=0){
+ // return;
+ // }
+ // shuffleIdAndWindowInstance2IsLocal.put(MapKeyUtil.createKey(splitNumer,windowInstanceId),2);
+ // executorService.execute(new Runnable() {
+ // @Override
+ // public void run() {
+ // if (rowOperator == null) {
+ // dbStorage.loadSplitData2Local(splitNumer, windowNameSpace, windowName, startTime, endOrFireTime,
+ // clazz,
+ // new IRowOperator() {
+ // @Override
+ // public void doProcess(Map<String, Object> row) {
+ // WindowBaseValue theValue = ORMUtil.convert(row, clazz);
+ // List<String> keys = new ArrayList<>();
+ // keys.add(theValue.getMsgKey());
+ // WindowBaseValue windowBaseValue = (WindowBaseValue)rocksDBStorage.get(keys, clazz);
+ // if (windowBaseValue == null) {
+ // Map<String, WindowBaseValue> map = new HashMap<>();
+ // map.put(theValue.getMsgKey(), theValue);
+ // rocksDBStorage.put(map, true);
+ // return;
+ // }
+ // if (theValue.getUpdateVersion() > windowBaseValue.getUpdateVersion()) {
+ // Map<String, WindowBaseValue> map = new HashMap<>();
+ // map.put(theValue.getMsgKey(), theValue);
+ // rocksDBStorage.put(map, true);
+ // }
+ // }
+ // });
+ // } else {
+ // dbStorage.loadSplitData2Local(splitNumer, windowNameSpace, windowName, startTime, endOrFireTime,
+ // clazz, rowOperator);
+ // }
+ // shuffleIdAndWindowInstance2IsLocal.put(MapKeyUtil.createKey(splitNumer, windowInstanceId), 1);
+ // }
+ // });
+ // }
+ //
+ //
+ //}
+ //
+ //
+ //@Override
+ //public boolean isLocalStorage(String shuffleId,String windowInstanceId) {
+ // Integer value=this.shuffleIdAndWindowInstance2IsLocal.get(MapKeyUtil.createKey(shuffleId,windowInstanceId));
+ // return isLocalStorage(value)||isLocalOnly;
+ //}
+ //@Override
+ //public void setLocalStorageInvalid(ISplit channelQueue,String windowInstanceId) {
+ // this.shuffleIdAndWindowInstance2IsLocal.remove(MapKeyUtil.createKey(channelQueue.getQueueId(),windowInstanceId),false);
+ //}
+ //@Override
+ //public void setLocalStorageInvalid(ISplit channelQueue) {
+ // Iterator<Entry<String, Integer>> it = this.shuffleIdAndWindowInstance2IsLocal.entrySet().iterator();
+ // List<String> keys=new ArrayList<>();
+ // while (it.hasNext()){
+ // Entry<String, Integer> entry=it.next();
+ // String key=entry.getKey();
+ // if(key.startsWith(channelQueue.getQueueId())){
+ // this.shuffleIdAndWindowInstance2IsLocal.put(key,0);
+ // keys.add(entry.getKey());
+ // }
+ //
+ // }
+ // executorService.execute(new Runnable() {
+ // @Override
+ // public void run() {
+ // clearCache(channelQueue);
+ // for(String key:keys){
+ // shuffleIdAndWindowInstance2IsLocal.remove(key);
+ // }
+ // }
+ // });
+ //}
+ //
+ //@Override
+ //public boolean setLocalStorageValdateIfNotExist(String shuffleId,String windowInstanceId) {
+ // Integer value = this.shuffleIdAndWindowInstance2IsLocal.get(
+ // MapKeyUtil.createKey(shuffleId, windowInstanceId));
+ // if (value != null) {
+ // return isLocalStorage(value);
+ // }
+ // this.shuffleIdAndWindowInstance2IsLocal.put( MapKeyUtil.createKey(shuffleId, windowInstanceId), 1);
+ // return true;
+ //}
+ //
+ //
+ //@Override
+ //public void clearWindowInstanceStorageStatus(String windowInstanceId,Collection<String> queueIds){
+ // if(queueIds!=null){
+ // for(String queueId:queueIds){
+ // this.shuffleIdAndWindowInstance2IsLocal.remove(MapKeyUtil.createKey(queueId,windowInstanceId));
+ // }
+ // }
+ //}
+ //
+ //
+ //public boolean isLocalStorage(Integer value){
+ // if(value!=null&&value==1){
+ // return true;
+ // }
+ // if(isLocalOnly){
+ // return true;
+ // }
+ // return false;
+ //}
+ //
+ //public boolean isLocalOnly() {
+ // return isLocalOnly;
+ //}
+ //
+ //public void setLocalOnly(boolean localOnly) {
+ // isLocalOnly = localOnly;
+ //}
+}
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
new file mode 100644
index 0000000..a3eff00
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java
@@ -0,0 +1,183 @@
+/*
+ * 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.storage;
+
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
+import org.apache.rocketmq.streams.window.state.WindowBaseValue;
+import org.apache.rocketmq.streams.window.storage.db.DBStorage;
+import org.apache.rocketmq.streams.window.storage.rocksdb.RocksdbStorage;
+
+import java.util.*;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+public class WindowStorage<T extends WindowBaseValue> extends AbstractWindowStorage<T> {
+ protected transient ShufflePartitionManager shufflePartitionManager = ShufflePartitionManager.getInstance();
+ protected IWindowStorage localStorage;
+ protected IWindowStorage remoteStorage;
+
+ private ExecutorService executorService;
+
+ //private ExecutorService dbService;
+ public WindowStorage(boolean isLoaclStorageOnly) {
+ this();
+ this.isLocalStorageOnly = isLoaclStorageOnly;
+ }
+
+ public WindowStorage() {
+ localStorage = new RocksdbStorage();
+ remoteStorage = new DBStorage();
+ executorService = new ThreadPoolExecutor(10, 10,
+ 0L, TimeUnit.MILLISECONDS,
+ new LinkedBlockingQueue<Runnable>());
+ //dbService= new ThreadPoolExecutor(1, 1,
+ // 0L, TimeUnit.MILLISECONDS,
+ // new LinkedBlockingQueue<Runnable>());
+
+ }
+
+ @Override
+ public WindowBaseValueIterator<T> loadWindowInstanceSplitData(String localStorePrefix, String queueId, String windowInstanceId, String keyPrefix,
+ Class<T> clazz) {
+ if (isLocalStorageOnly) {
+ return localStorage.loadWindowInstanceSplitData(localStorePrefix, queueId, windowInstanceId, keyPrefix, clazz);
+ }
+ if (shufflePartitionManager.isWindowInstanceFinishInit(queueId, windowInstanceId)) {
+ return localStorage.loadWindowInstanceSplitData(localStorePrefix, queueId, windowInstanceId, keyPrefix, clazz);
+ }
+ return remoteStorage.loadWindowInstanceSplitData(localStorePrefix, queueId, windowInstanceId, keyPrefix
+ , clazz);
+ }
+
+ protected transient ExecutorService executor = new ThreadPoolExecutor(10, 10,
+ 0L, TimeUnit.MILLISECONDS,
+ new LinkedBlockingQueue<Runnable>(100));
+
+ @Override
+ public void multiPut(Map<String, T> values, String windowInstanceId, String queueId) {
+ localStorage.multiPut(values);
+ if (isLocalStorageOnly) {
+ return;
+ }
+ if (shufflePartitionManager.isWindowInstanceFinishInit(queueId, windowInstanceId)) {
+ //可以考虑异步
+
+ remoteStorage.multiPut(values);
+
+ return;
+ }
+ remoteStorage.multiPut(values);
+ }
+
+ @Override
+ public Map<String, T> multiGet(Class<T> clazz, List<String> keys, String windowInstanceId, String queueId) {
+ if (isLocalStorageOnly || shufflePartitionManager.isWindowInstanceFinishInit(queueId, windowInstanceId)) {
+ return localStorage.multiGet(clazz, keys);
+ }
+ return remoteStorage.multiGet(clazz, keys);
+ }
+
+ @Override
+ public void multiPut(Map<String, T> values) {
+ localStorage.multiPut(values);
+ if (isLocalStorageOnly) {
+ return;
+ }
+ remoteStorage.multiPut(values);
+
+ }
+
+ @Override
+ public Map<String, T> multiGet(Class<T> clazz, List<String> keys) {
+ Map<String, T> result = new HashMap<>();
+ if (isLocalStorageOnly) {
+ result.putAll(localStorage.multiGet(clazz, keys));
+ return result;
+ }
+
+ List<String> notLocalKeys = new ArrayList<>();
+ List<String> 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));
+ return result;
+ }
+
+ @Override
+ public void removeKeys(Collection<String> keys) {
+ localStorage.removeKeys(keys);
+ }
+
+ /**
+ * refer to: WindowMessageProcessor.createStoreKey
+ */
+ public static String createWindowInstanceId(String msgKey) {
+ String[] values = MapKeyUtil.spliteKey(msgKey);
+ String[] lastValues = Arrays.copyOfRange(values, 1, values.length - 2);
+ //values[4]: endTime or fireTime
+ return MapKeyUtil.createKey(lastValues);
+ }
+
+ @Override
+ public void delete(String windowInstanceId, Set<String> queueIds, Class<T> clazz) {
+ executorService.execute(new Runnable() {
+
+ @Override
+ public void run() {
+ localStorage.delete(windowInstanceId, queueIds, clazz);
+ if (!isLocalStorageOnly) {
+ remoteStorage.delete(windowInstanceId, queueIds, clazz);
+ }
+
+ }
+ });
+ }
+
+ public static abstract class WindowBaseValueIterator<T extends WindowBaseValue> implements Iterator<T> {
+ protected long partitionNum = -1;
+
+ public void setPartitionNum(long partitionNum) {
+ this.partitionNum = partitionNum;
+ }
+ }
+
+ @Override
+ public void clearCache(ISplit split, Class<T> clazz) {
+ localStorage.clearCache(split, clazz);
+ }
+
+ public IWindowStorage getLocalStorage() {
+ return localStorage;
+ }
+
+ public IWindowStorage getRemoteStorage() {
+ return remoteStorage;
+ }
+}
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
new file mode 100644
index 0000000..3d2b2e2
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java
@@ -0,0 +1,186 @@
+/*
+ * 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.storage.db;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.state.WindowBaseValue;
+import org.apache.rocketmq.streams.window.storage.AbstractWindowStorage;
+import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator;
+
+/**
+ * database storage
+ */
+public class DBStorage<T extends WindowBaseValue> extends AbstractWindowStorage<T> {
+
+ @Override
+ public void multiPut(Map<String, T> values) {
+ if (CollectionUtil.isEmpty(values)) {
+ return;
+ }
+ ORMUtil.batchReplaceInto(values.values());
+ }
+
+ @Override
+ public Map<String, T> multiGet(Class<T> clazz, List<String> keys) {
+ if (CollectionUtil.isEmpty(keys)) {
+ return new HashMap<>(4);
+ }
+ Map<String, String> md5Key2Keys = new HashMap<>();
+ List<String> md5Keys = new ArrayList<>();
+ for (String key : keys) {
+ String md5Key = StringUtil.createMD5Str(key);
+ md5Keys.add(md5Key);
+ md5Key2Keys.put(md5Key, key);
+ }
+ List<T> values = ORMUtil.queryForList("select * from " + ORMUtil.getTableName(clazz) +
+ " where msg_key in (" + SQLUtil.createInSql(md5Keys) + " )", new HashMap<>(4), clazz);
+ Map<String, T> map = new HashMap<>(keys.size());
+ for (T value : values) {
+ String key = md5Key2Keys.get(value.getMsgKey());
+ map.put(key, value);
+ }
+ return map;
+ }
+
+ @Override
+ public void removeKeys(Collection<String> keys) {
+
+ }
+
+ @Override
+ public WindowBaseValueIterator<T> loadWindowInstanceSplitData(String localStorePrefix, String queueId, String windowInstanceId, String keyPrex, Class<T> clazz) {
+
+ //search max partition number in case of inserting fresh data [min,max)
+ long maxPartitionIndex = getPartitionNum(queueId, windowInstanceId, clazz, true) + 1;
+ long mimPartitionIndex = getPartitionNum(queueId, windowInstanceId, clazz, false) - 1;
+ if (maxPartitionIndex <= 1) {
+ return new WindowBaseValueIterator<T>() {
+ @Override
+ public boolean hasNext() {
+ return false;
+ }
+
+ @Override
+ public T next() {
+ return null;
+ }
+ };
+ }
+
+ DBIterator dbIterator = new DBIterator<T>(queueId, windowInstanceId, keyPrex, clazz, maxPartitionIndex);
+ dbIterator.setPartitionNum(mimPartitionIndex);
+ return dbIterator;
+ }
+
+ @Override
+ public void clearCache(ISplit channelQueue, Class<T> clazz) {
+ throw new RuntimeException("can not support this method");
+ }
+
+ @Override
+ public void delete(String windowInstanceId, Set<String> queueIds, Class<T> clazz) {
+ String sql = "delete from " + ORMUtil.getTableName(clazz) + " where window_instance_id = '" + StringUtil.createMD5Str(windowInstanceId) + "'";
+ ORMUtil.executeSQL(
+ sql,
+ new HashMap<>(4));
+ }
+
+ public static class DBIterator<T extends WindowBaseValue> extends WindowBaseValueIterator<T> {
+ private LinkedList<T> container = new LinkedList<>();
+ int batchSize = 1000;
+ private boolean exist = true;
+
+ private long maxPartitionIndex;
+ private Class<T> clazz;
+
+ String sql;
+
+ public DBIterator(String queueId, String windowInstanceId, String keyPrex, Class<T> clazz, long maxPartitionIndex) {
+ String windowInstancePartitionId = StringUtil.createMD5Str(windowInstanceId);
+
+ if (StringUtil.isEmpty(keyPrex)) {
+ sql = "select * from " + ORMUtil.getTableName(clazz)
+ + " where window_instance_partition_id = '" + windowInstancePartitionId
+ + "' and partition_num > #{partitionNum} order by window_instance_partition_id, partition_num limit "
+ + batchSize;
+ } else {
+ //join usage(different clazz)
+ String prefix = MapKeyUtil.createKey(queueId, windowInstanceId, keyPrex);
+ sql = "select * from " + ORMUtil.getTableName(clazz) + " where window_instance_partition_id ='"
+ + windowInstancePartitionId + "' " +
+ "and msg_key like '" + prefix
+ + "%' and partition_num > #{partitionNum} order by window_instance_partition_id, partition_num limit "
+ + batchSize;
+ }
+ this.maxPartitionIndex = maxPartitionIndex;
+ this.clazz = clazz;
+ }
+
+ @Override
+ public boolean hasNext() {
+ if (!container.isEmpty()) {
+ return true;
+ } else if (!exist) {
+ return false;
+ } else {
+ Map<String, Long> parameter = new HashMap<>(4);
+ parameter.put("partitionNum", partitionNum);
+ exist = partitionNum + batchSize <= maxPartitionIndex;
+ List<T> batchResult = ORMUtil.queryForList(sql, parameter, clazz);
+ if (CollectionUtil.isEmpty(batchResult)) {
+ return false;
+ } else {
+ partitionNum = batchResult.get(batchResult.size() - 1).getPartitionNum();
+ container.addAll(batchResult);
+ return true;
+ }
+ }
+ }
+
+ @Override
+ public T next() {
+ return container.poll();
+ }
+
+ }
+
+ protected Long getPartitionNum(String queueId, String windowInstanceId, Class<T> clazz, boolean isMax) {
+ String partitionNumSQL = isMax ? "max(partition_num)" : "min(partition_num)";
+ String windowInstancePartitionId = StringUtil.createMD5Str(windowInstanceId);
+ String sql = "select " + partitionNumSQL + " as partition_num from " + ORMUtil.getTableName(clazz)
+ + " where window_instance_partition_id ='" + windowInstancePartitionId + "'";
+ WindowBaseValue windowBaseValue = ORMUtil.queryForObject(sql, new HashMap<>(4), clazz);
+ if (windowBaseValue == null) {
+ return null;
+ }
+ return windowBaseValue.getPartitionNum();
+ }
+
+}
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
new file mode 100644
index 0000000..83dd299
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/file/FileStorage.java
@@ -0,0 +1,196 @@
+/*
+ * 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.storage.file;
+
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+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.StringUtil;
+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.WindowStorage.WindowBaseValueIterator;
+
+import java.util.*;
+import java.util.Map.Entry;
+
+public class FileStorage<T extends WindowBaseValue> extends AbstractWindowStorage<T> {
+ private static final String SPLIT_SIGN = "############";
+ protected transient String filePath = "/tmp/storage/file.storage";
+ protected transient Map<String, String> cache = new HashMap<>();
+
+ @Override
+ public synchronized void clearCache(ISplit channelQueue, Class<T> clazz) {
+ String queueId = channelQueue.getQueueId();
+ deleteByKeyPrefix(queueId);
+ this.flush();
+ }
+
+ @Override
+ public synchronized void delete(String windowInstanceId, Set<String> queueIds, Class<T> clazz) {
+ Set<String> currentQueueIds = new HashSet<>(queueIds);
+ Iterator<String> it = currentQueueIds.iterator();
+ while (it.hasNext()) {
+ String queueId = it.next();
+ String firstKey = MapKeyUtil.createKey(queueId, windowInstanceId);
+ deleteByKeyPrefix(firstKey);
+ }
+ this.flush();
+ }
+
+ @Override
+ public synchronized WindowBaseValueIterator<T> loadWindowInstanceSplitData(String localStorePrefix, String queueId,
+ String windowInstanceId, String key, Class<T> clazz) {
+ String keyPrefix = MapKeyUtil.createKey(queueId, windowInstanceId, key);
+ if (StringUtil.isNotEmpty(localStorePrefix)) {
+ keyPrefix = localStorePrefix + keyPrefix;
+ }
+ final String keyPrefixFinnal = keyPrefix;
+ Map<String, String> copyCache = new HashMap<>();
+ copyCache.putAll(this.cache);
+ final Iterator<Entry<String, String>> iter = copyCache.entrySet().iterator();
+ return new WindowBaseValueIterator<T>() {
+
+ @Override
+ public boolean hasNext() {
+ return iter.hasNext();
+ }
+
+ @Override
+ public T next() {
+ while (iter.hasNext()) {
+ Entry<String, String> entry = iter.next();
+ String key = entry.getKey();
+ String value = entry.getValue();
+ if (key.startsWith(keyPrefixFinnal)) {
+ T jsonable = ReflectUtil.forInstance(clazz);
+ jsonable.toObject(value);
+ return jsonable;
+ }
+ }
+ return null;
+ }
+ };
+ }
+
+ @Override
+ public synchronized void multiPut(Map<String, T> map) {
+ for (Entry<String, T> entry : map.entrySet()) {
+ String key = entry.getKey();
+ String value = entry.getValue().toJson();
+ this.cache.put(key, value);
+ }
+ this.flush();
+ }
+
+ @Override
+ public synchronized Map<String, T> multiGet(Class<T> clazz, List<String> keys) {
+ Map<String, T> result = new HashMap<>();
+ for (String key : keys) {
+ String value = this.cache.get(key);
+ if (StringUtil.isNotEmpty(value)) {
+ T jsonable = ReflectUtil.forInstance(clazz);
+ jsonable.toObject(value);
+ result.put(key, jsonable);
+ }
+ }
+ return result;
+ }
+
+ @Override
+ public synchronized void removeKeys(Collection<String> keys) {
+ for (String key : keys) {
+ this.cache.remove(key);
+ }
+ this.flush();
+ }
+
+ protected synchronized void deleteByKeyPrefix(String keyPrefix) {
+ Map<String, String> copyCache = new HashMap<>();
+ copyCache.putAll(this.cache);
+ Iterator<Entry<String, String>> it = copyCache.entrySet().iterator();
+ while (it.hasNext()) {
+ Entry<String, String> entry = it.next();
+ String key = entry.getKey();
+ if (key.startsWith(keyPrefix)) {
+ cache.remove(key);
+ }
+ }
+ }
+
+ private synchronized void flush() {
+ List<String> buffer = new ArrayList<>();
+ Map<String, String> copyCache = new HashMap<>();
+ copyCache.putAll(this.cache);
+
+ Iterator<Entry<String, String>> it = copyCache.entrySet().iterator();
+ while (it.hasNext()) {
+ Entry<String, String> entry = it.next();
+ String key = entry.getKey();
+ String value = entry.getValue();
+ String line = MapKeyUtil.createKeyBySign(SPLIT_SIGN, key, value);
+ buffer.add(line);
+ }
+ FileUtil.write(filePath, buffer, false);
+ }
+
+ private void load() {
+ Map<String, String> cache = new HashMap<>();
+ List<String> buffer = FileUtil.loadFileLine(filePath);
+ for (String line : buffer) {
+ String[] values = line.split(SPLIT_SIGN);
+ cache.put(values[0], values[1]);
+ }
+ this.cache = cache;
+ }
+
+ //_order_by_split_num_1;1;namespace;name_window_10001;2021-07-13 15:07:40;2021-07-13 02:35:10;2021-07-13 02:35:15
+
+ public static void main(String[] args) {
+ FileStorage fileStorage = new FileStorage();
+ fileStorage.load();
+ fileStorage.deleteByKeyPrefix("_order_by_split_num_1;1;namespace;name_window_10001;2021-07-13 15:07:40;2021-07-13 02:35:10;2021-07-13 02:35:15");
+ fileStorage.deleteByKeyPrefix("1;1;namespace;name_window_10001;2021-07-13 15:07:40;2021-07-13 02:35:10;2021-07-13 02:35:15");
+ fileStorage.deleteByKeyPrefix("1;1;namespace;name_window_10001;2021-07-13 15:07:40;2021-07-13 02:35:05;2021-07-13 02:35:10");
+ fileStorage.deleteByKeyPrefix(" _order_by_split_num_1;1;namespace;name_window_10001;2021-07-13 15:07:40;2021-07-13 02:35:05;2021-07-13 02:35:10");
+ fileStorage.flush();
+ fileStorage.load();
+ WindowBaseValueIterator<WindowValue> fileIterator = fileStorage.loadWindowInstanceSplitData("_order_by_split_num_", "1", "1;namespace;name_window_10001",
+ null, WindowValue.class);
+ int sum = 0;
+ while (fileIterator.hasNext()) {
+ WindowValue windowValue = fileIterator.next();
+ if (windowValue == null) {
+ break;
+ }
+ sum++;
+ }
+
+ fileIterator = fileStorage.loadWindowInstanceSplitData(null, "1", "1;namespace;name_window_10001",
+ null, WindowValue.class);
+ while (fileIterator.hasNext()) {
+ WindowValue windowValue = fileIterator.next();
+ if (windowValue == null) {
+ break;
+ }
+ sum++;
+ }
+ System.out.println(sum);
+
+ }
+}
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
new file mode 100644
index 0000000..a603302
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java
@@ -0,0 +1,293 @@
+/*
+ * 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.storage.rocksdb;
+
+import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import org.apache.rocketmq.streams.common.utils.*;
+import org.apache.rocketmq.streams.window.state.WindowBaseValue;
+import org.apache.rocketmq.streams.window.storage.AbstractWindowStorage;
+import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator;
+import org.rocksdb.*;
+
+import java.io.File;
+import java.io.UnsupportedEncodingException;
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class RocksdbStorage<T extends WindowBaseValue> extends AbstractWindowStorage<T> {
+ 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 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<String> keys) {
+
+ for (String key : keys) {
+ try {
+ rocksDB.delete(getKeyBytes(key));
+ } catch (RocksDBException e) {
+ throw new RuntimeException("delete error " + key);
+ }
+ }
+
+ }
+
+ @Override
+ public WindowBaseValueIterator<T> loadWindowInstanceSplitData(String localStorePrefix, String queueId, String windowInstanceId, String key, Class<T> clazz) {
+ String keyPrefix = MapKeyUtil.createKey(queueId, windowInstanceId, key);
+ if (StringUtil.isNotEmpty(localStorePrefix)) {
+ keyPrefix = localStorePrefix + keyPrefix;
+ }
+ return getByKeyPrefix(keyPrefix, clazz, false);
+ }
+
+ @Override
+ public void multiPut(Map<String, T> values) {
+ if (values == null) {
+ return;
... 6737 lines suppressed ...