Repository: apache/rocketmq-streams Branch: develop Commit: 22976de3ef0f Files: 132 Total size: 452.2 KB Directory structure: gitextract_86qvi7ak/ ├── .asf.yaml ├── .gitignore ├── .travis.yml ├── LICENSE ├── NOTICE ├── README.md ├── build_without_test.sh ├── core/ │ ├── pom.xml │ └── src/ │ ├── main/ │ │ ├── java/ │ │ │ └── org/ │ │ │ └── apache/ │ │ │ └── rocketmq/ │ │ │ └── streams/ │ │ │ └── core/ │ │ │ ├── RocketMQStream.java │ │ │ ├── common/ │ │ │ │ └── Constant.java │ │ │ ├── exception/ │ │ │ │ ├── DataProcessThrowable.java │ │ │ │ ├── DeserializeThrowable.java │ │ │ │ ├── RStreamsException.java │ │ │ │ └── RecoverStateStoreThrowable.java │ │ │ ├── function/ │ │ │ │ ├── AggregateAction.java │ │ │ │ ├── FilterAction.java │ │ │ │ ├── ForeachAction.java │ │ │ │ ├── SelectAction.java │ │ │ │ ├── ValueJoinAction.java │ │ │ │ ├── ValueMapperAction.java │ │ │ │ ├── accumulator/ │ │ │ │ │ ├── Accumulator.java │ │ │ │ │ ├── AvgAccumulator.java │ │ │ │ │ ├── CountAccumulator.java │ │ │ │ │ └── MinAccumulator.java │ │ │ │ └── supplier/ │ │ │ │ ├── AccumulatorSupplier.java │ │ │ │ ├── AddTagSupplier.java │ │ │ │ ├── AggregateSupplier.java │ │ │ │ ├── FilterSupplier.java │ │ │ │ ├── ForeachSupplier.java │ │ │ │ ├── JoinAggregateSupplier.java │ │ │ │ ├── JoinWindowAggregateSupplier.java │ │ │ │ ├── KeySelectSupplier.java │ │ │ │ ├── MultiValueChangeSupplier.java │ │ │ │ ├── PrintSupplier.java │ │ │ │ ├── SinkSupplier.java │ │ │ │ ├── SourceSupplier.java │ │ │ │ ├── SumAggregate.java │ │ │ │ ├── TimestampSelectorSupplier.java │ │ │ │ ├── ValueChangeSupplier.java │ │ │ │ ├── WindowAccumulatorSupplier.java │ │ │ │ └── WindowAggregateSupplier.java │ │ │ ├── metadata/ │ │ │ │ ├── Data.java │ │ │ │ └── StreamConfig.java │ │ │ ├── rstream/ │ │ │ │ ├── GroupedStream.java │ │ │ │ ├── GroupedStreamImpl.java │ │ │ │ ├── JoinedStream.java │ │ │ │ ├── Pipeline.java │ │ │ │ ├── RStream.java │ │ │ │ ├── RStreamImpl.java │ │ │ │ ├── StreamBuilder.java │ │ │ │ ├── WindowStream.java │ │ │ │ └── WindowStreamImpl.java │ │ │ ├── running/ │ │ │ │ ├── AbstractProcessor.java │ │ │ │ ├── AbstractWindowProcessor.java │ │ │ │ ├── MessageQueueListenerWrapper.java │ │ │ │ ├── Processor.java │ │ │ │ ├── RocketMQClient.java │ │ │ │ ├── StreamContext.java │ │ │ │ ├── StreamContextImpl.java │ │ │ │ └── WorkerThread.java │ │ │ ├── serialization/ │ │ │ │ ├── KeyValueDeserializer.java │ │ │ │ ├── KeyValueSerializer.java │ │ │ │ ├── ShuffleProtocol.java │ │ │ │ ├── deImpl/ │ │ │ │ │ └── KVJsonDeserializer.java │ │ │ │ └── serImpl/ │ │ │ │ └── KVJsonSerializer.java │ │ │ ├── state/ │ │ │ │ ├── AbstractStore.java │ │ │ │ ├── RocketMQStore.java │ │ │ │ ├── RocksDBStore.java │ │ │ │ └── StateStore.java │ │ │ ├── topology/ │ │ │ │ ├── TopologyBuilder.java │ │ │ │ ├── real/ │ │ │ │ │ ├── ProcessorFactory.java │ │ │ │ │ ├── RealProcessorFactory.java │ │ │ │ │ ├── SinkFactory.java │ │ │ │ │ └── SourceFactory.java │ │ │ │ └── virtual/ │ │ │ │ ├── AbstractGraphNode.java │ │ │ │ ├── GraphNode.java │ │ │ │ ├── ProcessorNode.java │ │ │ │ ├── ShuffleProcessorNode.java │ │ │ │ ├── SinkGraphNode.java │ │ │ │ └── SourceGraphNode.java │ │ │ ├── util/ │ │ │ │ ├── ColumnFamilyUtil.java │ │ │ │ ├── OperatorNameMaker.java │ │ │ │ ├── Pair.java │ │ │ │ ├── RocketMQUtil.java │ │ │ │ └── Utils.java │ │ │ └── window/ │ │ │ ├── JoinType.java │ │ │ ├── StreamType.java │ │ │ ├── Time.java │ │ │ ├── TimeType.java │ │ │ ├── Window.java │ │ │ ├── WindowBuilder.java │ │ │ ├── WindowInfo.java │ │ │ ├── WindowKey.java │ │ │ ├── WindowState.java │ │ │ ├── WindowStore.java │ │ │ └── fire/ │ │ │ ├── AbstractWindowFire.java │ │ │ ├── AccumulatorSessionWindowFire.java │ │ │ ├── AccumulatorWindowFire.java │ │ │ ├── AggregateSessionWindowFire.java │ │ │ ├── AggregateWindowFire.java │ │ │ ├── IdleWindowScaner.java │ │ │ ├── JoinWindowFire.java │ │ │ └── WindowFire.java │ │ └── resources/ │ │ └── logback.xml │ └── test/ │ └── java/ │ └── org/ │ └── apache/ │ └── rocketmq/ │ └── streams/ │ └── core/ │ ├── Num.java │ ├── Test.java │ ├── User.java │ ├── function/ │ │ └── AggregateSupplier.java │ ├── runtime/ │ │ └── WindowStateTests.java │ ├── state/ │ │ └── RocksDBStoreTest.java │ └── util/ │ └── UtilsTests.java ├── example/ │ ├── pom.xml │ └── src/ │ └── main/ │ └── java/ │ └── org/ │ └── apache/ │ └── rocketmq/ │ └── streams/ │ └── examples/ │ ├── WordCount.java │ ├── WordCountFromFirstOffset.java │ ├── joinWindow/ │ │ ├── JoinWindow.java │ │ └── JoinWithoutWindow.java │ ├── pojo/ │ │ ├── Demo.java │ │ ├── Grade.java │ │ ├── Num.java │ │ ├── Order.java │ │ ├── Union.java │ │ └── User.java │ ├── sink/ │ │ └── WordCountSink.java │ └── window/ │ ├── SessionWindowCount.java │ ├── SlideWindowCount.java │ ├── WindowAvg.java │ ├── WindowCount.java │ ├── WindowMin.java │ ├── WindowOrderCount.java │ ├── WindowSum.java │ └── WindowWordCount.java └── pom.xml ================================================ FILE CONTENTS ================================================ ================================================ FILE: .asf.yaml ================================================ github: features: # Enable issue management issues: true # Enable wiki wiki: true protected_branches: notifications: commits: commits@rocketmq.apache.org issues: commits@rocketmq.apache.org pullrequests: commits@rocketmq.apache.org jobs: commits@rocketmq.apache.org discussions: dev@rocketmq.apache.org ================================================ FILE: .gitignore ================================================ target/ .DS_Store !.mvn/wrapper/maven-wrapper.jar *.versionsBackup .gradle/ ### STS ### .apt_generated .classpath .factorypath .project .settings .springBeans ### IntelliJ IDEA ### .idea *.iws *.iml *.ipr *.zip ### NetBeans ### nbproject/private/ build/ nbbuild/ dist/ nbdist/ pom.xml.releaseBackup release.properties ================================================ FILE: .travis.yml ================================================ dist: trusty language: java jdk: - oraclejdk8 - oraclejdk9 - oraclejdk11 - openjdk8 before_install: - echo 'MAVEN_OPTS="$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal"' >> ~/.mavenrc - cat ~/.mavenrc - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi - if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "$CUSTOM_JDK"; fi script: - travis_retry mvn -B clean apache-rat:check - travis_retry mvn -B clean install cobertura:cobertura after_success: - bash <(curl -s https://codecov.io/bash) || echo 'Codecov failed to upload' ================================================ FILE: LICENSE ================================================ Apache License Version 2.0, January 2004 http://www.apache.org/licenses/ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 1. Definitions. "License" shall mean the terms and conditions for use, reproduction, and distribution as defined by Sections 1 through 9 of this document. "Licensor" shall mean the copyright owner or entity authorized by the copyright owner that is granting the License. "Legal Entity" shall mean the union of the acting entity and all other entities that control, are controlled by, or are under common control with that entity. For the purposes of this definition, "control" means (i) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the outstanding shares, or (iii) beneficial ownership of such entity. "You" (or "Your") shall mean an individual or Legal Entity exercising permissions granted by this License. "Source" form shall mean the preferred form for making modifications, including but not limited to software source code, documentation source, and configuration files. "Object" form shall mean any form resulting from mechanical transformation or translation of a Source form, including but not limited to compiled object code, generated documentation, and conversions to other media types. "Work" shall mean the work of authorship, whether in Source or Object form, made available under the License, as indicated by a copyright notice that is included in or attached to the work (an example is provided in the Appendix below). "Derivative Works" shall mean any work, whether in Source or Object form, that is based on (or derived from) the Work and for which the editorial revisions, annotations, elaborations, or other modifications represent, as a whole, an original work of authorship. For the purposes of this License, Derivative Works shall not include works that remain separable from, or merely link (or bind by name) to the interfaces of, the Work and Derivative Works thereof. "Contribution" shall mean any work of authorship, including the original version of the Work and any modifications or additions to that Work or Derivative Works thereof, that is intentionally submitted to Licensor for inclusion in the Work by the copyright owner or by an individual or Legal Entity authorized to submit on behalf of the copyright owner. For the purposes of this definition, "submitted" means any form of electronic, verbal, or written communication sent to the Licensor or its representatives, including but not limited to communication on electronic mailing lists, source code control systems, and issue tracking systems that are managed by, or on behalf of, the Licensor for the purpose of discussing and improving the Work, but excluding communication that is conspicuously marked or otherwise designated in writing by the copyright owner as "Not a Contribution." "Contributor" shall mean Licensor and any individual or Legal Entity on behalf of whom a Contribution has been received by Licensor and subsequently incorporated within the Work. 2. Grant of Copyright License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable copyright license to reproduce, prepare Derivative Works of, publicly display, publicly perform, sublicense, and distribute the Work and such Derivative Works in Source or Object form. 3. Grant of Patent License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable (except as stated in this section) patent license to make, have made, use, offer to sell, sell, import, and otherwise transfer the Work, where such license applies only to those patent claims licensable by such Contributor that are necessarily infringed by their Contribution(s) alone or by combination of their Contribution(s) with the Work to which such Contribution(s) was submitted. If You institute patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Work or a Contribution incorporated within the Work constitutes direct or contributory patent infringement, then any patent licenses granted to You under this License for that Work shall terminate as of the date such litigation is filed. 4. Redistribution. You may reproduce and distribute copies of the Work or Derivative Works thereof in any medium, with or without modifications, and in Source or Object form, provided that You meet the following conditions: (a) You must give any other recipients of the Work or Derivative Works a copy of this License; and (b) You must cause any modified files to carry prominent notices stating that You changed the files; and (c) You must retain, in the Source form of any Derivative Works that You distribute, all copyright, patent, trademark, and attribution notices from the Source form of the Work, excluding those notices that do not pertain to any part of the Derivative Works; and (d) If the Work includes a "NOTICE" text file as part of its distribution, then any Derivative Works that You distribute must include a readable copy of the attribution notices contained within such NOTICE file, excluding those notices that do not pertain to any part of the Derivative Works, in at least one of the following places: within a NOTICE text file distributed as part of the Derivative Works; within the Source form or documentation, if provided along with the Derivative Works; or, within a display generated by the Derivative Works, if and wherever such third-party notices normally appear. The contents of the NOTICE file are for informational purposes only and do not modify the License. You may add Your own attribution notices within Derivative Works that You distribute, alongside or as an addendum to the NOTICE text from the Work, provided that such additional attribution notices cannot be construed as modifying the License. You may add Your own copyright statement to Your modifications and may provide additional or different license terms and conditions for use, reproduction, or distribution of Your modifications, or for any such Derivative Works as a whole, provided Your use, reproduction, and distribution of the Work otherwise complies with the conditions stated in this License. 5. Submission of Contributions. Unless You explicitly state otherwise, any Contribution intentionally submitted for inclusion in the Work by You to the Licensor shall be under the terms and conditions of this License, without any additional terms or conditions. Notwithstanding the above, nothing herein shall supersede or modify the terms of any separate license agreement you may have executed with Licensor regarding such Contributions. 6. Trademarks. This License does not grant permission to use the trade names, trademarks, service marks, or product names of the Licensor, except as required for reasonable and customary use in describing the origin of the Work and reproducing the content of the NOTICE file. 7. Disclaimer of Warranty. Unless required by applicable law or agreed to in writing, Licensor provides the Work (and each Contributor provides its Contributions) on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied, including, without limitation, any warranties or conditions of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A PARTICULAR PURPOSE. You are solely responsible for determining the appropriateness of using or redistributing the Work and assume any risks associated with Your exercise of permissions under this License. 8. Limitation of Liability. In no event and under no legal theory, whether in tort (including negligence), contract, or otherwise, unless required by applicable law (such as deliberate and grossly negligent acts) or agreed to in writing, shall any Contributor be liable to You for damages, including any direct, indirect, special, incidental, or consequential damages of any character arising as a result of this License or out of the use or inability to use the Work (including but not limited to damages for loss of goodwill, work stoppage, computer failure or malfunction, or any and all other commercial damages or losses), even if such Contributor has been advised of the possibility of such damages. 9. Accepting Warranty or Additional Liability. While redistributing the Work or Derivative Works thereof, You may choose to offer, and charge a fee for, acceptance of support, warranty, indemnity, or other liability obligations and/or rights consistent with this License. However, in accepting such obligations, You may act only on Your own behalf and on Your sole responsibility, not on behalf of any other Contributor, and only if You agree to indemnify, defend, and hold each Contributor harmless for any liability incurred by, or claims asserted against, such Contributor by reason of your accepting any such warranty or additional liability. END OF TERMS AND CONDITIONS APPENDIX: How to apply the Apache License to your work. To apply the Apache License to your work, attach the following boilerplate notice, with the fields enclosed by brackets "[]" replaced with your own identifying information. (Don't include the brackets!) The text should be enclosed in the appropriate comment syntax for the file format. We also recommend that a file or class name and description of purpose be included on the same "printed page" as the copyright notice for easier identification within third-party archives. Copyright [yyyy] [name of copyright owner] Licensed 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. ================================================ FILE: NOTICE ================================================ Apache RocketMQ Copyright 2016-2023 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). ================================================ FILE: README.md ================================================ # RocketMQ Streams [![Build Status](https://app.travis-ci.com/apache/rocketmq-streams.svg?branch=main)](https://app.travis-ci.com/apache/rocketmq-streams) [![CodeCov](https://codecov.io/gh/apache/rocketmq-stream/branch/main/graph/badge.svg)](https://app.codecov.io/gh/apache/rocketmq-streams) [![GitHub release](https://img.shields.io/badge/release-download-orange.svg)](https://github.com/apache/rocketmq-streams/releases) [![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html) [![Average time to resolve an issue](http://isitmaintained.com/badge/resolution/apache/rocketmq-streams.svg)](http://isitmaintained.com/project/apache/rocketmq-streams "Average time to resolve an issue") [![Percentage of issues still open](http://isitmaintained.com/badge/open/apache/rocketmq-streams.svg)](http://isitmaintained.com/project/apache/rocketmq-streams "Percentage of issues still open") [![Twitter Follow](https://img.shields.io/twitter/follow/ApacheRocketMQ?style=social)](https://twitter.com/intent/follow?screen_name=ApacheRocketMQ) **RocketMQ Streams is a lightweight stream processing framework, application gains the stream processing ability by depending on RocketMQ Streams as an SDK.** It offers a variety of features: * Function: * One-to-one transform function, such as: filter, map, foreach * Aggregate function, such as: sum, min, max, count, aggregate * Generating function, such as: flatMap * Group by aggregate and window aggregate * Join stream * Custom serialization ---------- ## Quick Start This paragraph guides you running a stream processing with RocketMQ Streams. ### Run RocketMQ 5.0 locally [RocketMQ quick-start](https://rocketmq.apache.org/docs/quick-start/) RocketMQ runs on all major operating systems and requires only a Java JDK version 8 or higher to be installed. To check, run `java -version`: ```shell $ java -version java version "1.8.0_121" ``` **1) Download RocketMQ** ```shell wget https://archive.apache.org/dist/rocketmq/5.0.0/rocketmq-all-5.0.0-bin-release.zip # Unpack the release $ unzip rocketmq-all-5.0.0-bin-release.zip $ cd rocketmq-all-5.0.0-bin-release/bin ``` **2) Start NameServer** NameServer will be listening at `0.0.0.0:9876`, make sure that the port is not used by others on the local machine, and then do as follows. ```shell ### start Name Server $ nohup sh mqnamesrv & ### check whether Name Server is successfully started $ tail -f ~/logs/rocketmqlogs/namesrv.log The Name Server boot success... ``` **2) Start Broker** ```shell ### start Broker $ nohup sh bin/mqbroker -n localhost:9876 & ### check whether Broker is successfully started, eg: Broker's IP is 192.168.1.2, Broker's name is broker-a $ tail -f ~/logs/rocketmqlogs/broker.log The broker[broker-a, 192.169.1.2:10911] boot success... ``` ### Build stream processing application **1) Build application in IDE** **2) Add RocketMQ Streams dependency** ```xml org.apache.rocketmq rocketmq-streams {current.version} ``` **3) Build stream processing application** * create topic in RocketMQ before start the stream processing. ```shell sh bin/mqadmin updateTopic -c ${clusterName} -t ${topicName} -r 8 -w 8 -n 127.0.0.1:9876 ``` NOTE: the default clusterName is DefaultCluster in this quick-start doc, changes it with your RocketMQ cluster. * add your stream processing code, The following is an example. more examples are [here](./example/src/main/java/org/apache/rocketmq/streams/examples). ```java public static void main(String[] args) { StreamBuilder builder = new StreamBuilder("wordCount"); builder.source("sourceTopic", total -> { String value = new String(total, StandardCharsets.UTF_8); return new Pair<>(null, value); }) .flatMap((ValueMapperAction>) value -> { String[] splits = value.toLowerCase().split("\\W+"); return Arrays.asList(splits); }) .keyBy(value -> value) .count() .toRStream() .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.put(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); final CountDownLatch latch = new CountDownLatch(1); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); latch.countDown(); } }); try { rocketMQStream.start(); latch.await(); } catch (final Throwable e) { System.exit(1); } System.exit(0); } ``` ================================================ FILE: build_without_test.sh ================================================ #!/bin/sh # 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 ato # # 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. mvn clean package -Dmaven.test.skip=true ================================================ FILE: core/pom.xml ================================================ rocketmq-streams-all org.apache.rocketmq 1.1.2-SNAPSHOT 4.0.0 rocketmq-streams jar 8 8 org.apache.rocketmq rocketmq-common ${rocketmq.version} org.apache.rocketmq rocketmq-client ${rocketmq.version} org.apache.rocketmq rocketmq-tools com.google.protobuf protobuf-java ${rocketmq.version} org.rocksdb rocksdbjni 7.6.0 commons-io commons-io 2.11.0 com.fasterxml.jackson.core jackson-databind 2.13.4.2 ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/RocketMQStream.java ================================================ package org.apache.rocketmq.streams.core; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RStreamsException; import org.apache.rocketmq.streams.core.metadata.StreamConfig; import org.apache.rocketmq.streams.core.running.WorkerThread; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; import java.util.Properties; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; public class RocketMQStream { private static final Logger logger = LoggerFactory.getLogger(RocketMQStream.class.getName()); private static final AtomicInteger index = new AtomicInteger(1); private final TopologyBuilder topologyBuilder; private final Properties properties; private final List workerThreads = new ArrayList<>(); private final AtomicBoolean started = new AtomicBoolean(false); private final ScheduledExecutorService executor = Executors.newScheduledThreadPool( StreamConfig.SCHEDULED_THREAD_NUM, r -> new Thread(r, "ScanIdleWindowThread_" + index.getAndIncrement())); public RocketMQStream(TopologyBuilder topologyBuilder, Properties properties) { this.topologyBuilder = topologyBuilder; this.properties = properties; } public synchronized void start() { String jobId = topologyBuilder.getJobId(); if (started.get()) { logger.info("RocketMQStream has been started, jobId=[{}].", jobId); return; } this.started.compareAndSet(false, true); //启动线程 try { int threadNum = StreamConfig.STREAMS_PARALLEL_THREAD_NUM; for (int i = 0; i < threadNum; i++) { String threadName = String.join("_", Constant.WORKER_THREAD_NAME, jobId, String.valueOf(i)); WorkerThread thread = new WorkerThread(threadName, topologyBuilder, this.properties, executor); thread.start(); workerThreads.add(thread); } } catch (Throwable t) { logger.error("start RocketMQStream error, jobId=[{}].", jobId, t); throw new RStreamsException(t); } } public void stop() { for (WorkerThread thread : workerThreads) { thread.shutdown(); } workerThreads.clear(); this.started.set(false); } public boolean isRunning() { return this.started.get(); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/common/Constant.java ================================================ /* * 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.core.common; public class Constant { public static final String SHUFFLE_KEY_CLASS_NAME = "shuffle.key.class.name"; public static final String SHUFFLE_VALUE_CLASS_NAME = "shuffle.value.class.name"; public final static String STATE_TOPIC_SUFFIX = "-stateTopic"; public static final String SHUFFLE_TOPIC_SUFFIX = "-shuffleTopic"; public static final String SKIP_DATA_ERROR = "skip_data_error"; public static final String SPLIT = "@"; public static final String EMPTY_BODY = "empty_body"; public static final String TRUE = "true"; public static final String SOURCE_TIMESTAMP = "source_timestamp"; public static final String STREAM_TAG = "stream_tag"; public static final String WINDOW_START_TIME = "window_start_time"; public static final String WINDOW_END_TIME = "window_end_time"; public static final String WORKER_THREAD_NAME = "ROCKETMQ_STREAMS"; public static final String STATIC_TOPIC_BROKER_NAME = "__syslo__global__"; public static final String WATERMARK_KEY = "watermark_key"; public static final Long DEFAULT_CONSUME_OFFSET = 0L; } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/DataProcessThrowable.java ================================================ /* * 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.core.exception; public class DataProcessThrowable extends Throwable { public DataProcessThrowable() { } public DataProcessThrowable(String message) { super(message); } public DataProcessThrowable(String message, Throwable cause) { super(message, cause); } public DataProcessThrowable(Throwable cause) { super(cause); } public DataProcessThrowable(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { super(message, cause, enableSuppression, writableStackTrace); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/DeserializeThrowable.java ================================================ /* * 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.core.exception; public class DeserializeThrowable extends Throwable { private static final long serialVersionUID = 2154421351264920776L; public DeserializeThrowable() { } public DeserializeThrowable(String message) { super(message); } public DeserializeThrowable(String message, Throwable cause) { super(message, cause); } public DeserializeThrowable(Throwable cause) { super(cause); } public DeserializeThrowable(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { super(message, cause, enableSuppression, writableStackTrace); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/RStreamsException.java ================================================ /* * 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.core.exception; public class RStreamsException extends RuntimeException { private static final long serialVersionUID = 6729806497659471678L; public RStreamsException() { } public RStreamsException(String message) { super(message); } public RStreamsException(String message, Throwable cause) { super(message, cause); } public RStreamsException(Throwable cause) { super(cause); } public RStreamsException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { super(message, cause, enableSuppression, writableStackTrace); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/RecoverStateStoreThrowable.java ================================================ /* * 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.core.exception; public class RecoverStateStoreThrowable extends Throwable { public RecoverStateStoreThrowable() { } public RecoverStateStoreThrowable(String message) { super(message); } public RecoverStateStoreThrowable(String message, Throwable cause) { super(message, cause); } public RecoverStateStoreThrowable(Throwable cause) { super(cause); } public RecoverStateStoreThrowable(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { super(message, cause, enableSuppression, writableStackTrace); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/AggregateAction.java ================================================ /* * 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.core.function; public interface AggregateAction { OUT calculate(K key, V value, OUT accumulator); } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/FilterAction.java ================================================ /* * 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.core.function; @FunctionalInterface public interface FilterAction { /** * * @param value the target to filter * @return pass if true; */ boolean apply(final T value); } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/ForeachAction.java ================================================ /* * 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.core.function; @FunctionalInterface public interface ForeachAction { void apply(T value); } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/SelectAction.java ================================================ /* * 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.core.function; @FunctionalInterface public interface SelectAction { OUT select(IN value); } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/ValueJoinAction.java ================================================ /* * 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.core.function; public interface ValueJoinAction { OUT apply(final V1 value1, final V2 value2); } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/ValueMapperAction.java ================================================ /* * 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.core.function; public interface ValueMapperAction { O convert(final T value) throws Throwable; } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/Accumulator.java ================================================ /* * 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.core.function.accumulator; import java.util.Properties; //因为需要序列化/反序列化这个类,所以必须给field生成setter/getter方法 public interface Accumulator { void addValue(V value); void merge(Accumulator other); /** * invoked after the window fired. * * @param context the attached properties after window fired. * @return the value. */ R result(Properties context); Accumulator clone(); } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/AvgAccumulator.java ================================================ /* * 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.core.function.accumulator; import java.util.Properties; public class AvgAccumulator implements Accumulator { private Double avg; private Integer num; @Override public void addValue(V value) { if (value instanceof Number) { Number number = (Number) value; Double valueToDouble = number.doubleValue(); if (avg == null) { avg = valueToDouble; num = 1; } else { avg = avg + (valueToDouble - avg) / (num + 1); num++; } } else { throw new IllegalArgumentException("Calculate avg, input is not a number. value=" + value); } } @Override public void merge(Accumulator other) { if (other instanceof AvgAccumulator) { AvgAccumulator otherAvgAccumulator = (AvgAccumulator) other; Integer numOther = otherAvgAccumulator.getNum(); Double avgOther = otherAvgAccumulator.getAvg(); avg = avg + numOther / (num + numOther) * (avgOther - avg); num = num + numOther; } else { throw new IllegalArgumentException("Merge avg, input is not a AvgAccumulator."); } } @Override public Double result(Properties context) { return avg; } @Override public Accumulator clone() { return new AvgAccumulator<>(); } public Double getAvg() { return avg; } public void setAvg(Double avg) { this.avg = avg; } public Integer getNum() { return num; } public void setNum(Integer num) { this.num = num; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/CountAccumulator.java ================================================ /* * 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.core.function.accumulator; import java.util.Properties; //因为需要序列化/反序列化这个类,所以必须给field生成setter/getter方法 public class CountAccumulator implements Accumulator { private Integer count = 0; @Override public void addValue(V value) { count += 1; } @Override public void merge(Accumulator other) { count += other.result(null); } @Override public Integer result(Properties context) { return count; } public Integer getCount() { return count; } public void setCount(Integer count) { this.count = count; } @Override public Accumulator clone() { CountAccumulator accumulator = new CountAccumulator<>(); accumulator.count = this.count; return accumulator; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/MinAccumulator.java ================================================ /* * 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.core.function.accumulator; import java.util.Properties; public class MinAccumulator implements Accumulator { private Number min; @Override public void addValue(V value) { if (value instanceof Number) { Number number = (Number) value; if (min == null) { min = number; } else { min = Math.min(min.doubleValue(), number.doubleValue()); } } else { throw new IllegalArgumentException("min but not a number. value=" + value); } } @Override public void merge(Accumulator other) { min = Math.min(min.doubleValue(), other.result(null)); } @Override public Double result(Properties context) { return min.doubleValue(); } @Override public Accumulator clone() { return new MinAccumulator<>(); } public Number getMin() { return min; } public void setMin(Number min) { this.min = min; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AccumulatorSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable; import org.apache.rocketmq.streams.core.function.accumulator.Accumulator; import org.apache.rocketmq.streams.core.function.SelectAction; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.state.StateStore; import org.apache.rocketmq.streams.core.util.ColumnFamilyUtil; import java.util.function.Supplier; public class AccumulatorSupplier implements Supplier> { private final String currentName; private final String parentName; private SelectAction selectAction; private Accumulator accumulator; public AccumulatorSupplier(String currentName, String parentName, SelectAction selectAction, Accumulator accumulator) { this.currentName = currentName; this.parentName = parentName; this.selectAction = selectAction; this.accumulator = accumulator; } @Override public Processor get() { return new AccumulatorProcessor(currentName, parentName, selectAction, accumulator); } private class AccumulatorProcessor extends AbstractProcessor { private final String currentName; private final String parentName; private StateStore stateStore; private MessageQueue stateTopicMessageQueue; private SelectAction selectAction; private Accumulator accumulator; public AccumulatorProcessor(String currentName, String parentName, SelectAction selectAction, Accumulator accumulator) { this.currentName = currentName; this.parentName = parentName; this.selectAction = selectAction; this.accumulator = accumulator; } @Override public void preProcess(StreamContext context) throws RecoverStateStoreThrowable { super.preProcess(context); this.stateStore = super.waitStateReplay(); String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX; this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId()); } @Override public void process(V data) throws Throwable { K key = this.context.getKey(); Accumulator value; byte[] keyBytes = super.object2Byte(key); byte[] valueBytes = stateStore.get(ColumnFamilyUtil.VALUE_STATE_CF, keyBytes); if (valueBytes == null || valueBytes.length == 0) { value = accumulator.clone(); } else { value = super.byte2Object(valueBytes); } R select = selectAction.select(data); value.addValue(select); OV result = value.result(null); byte[] newValueBytes = super.object2Byte(value); stateStore.put(this.stateTopicMessageQueue, ColumnFamilyUtil.VALUE_STATE_CF, keyBytes, newValueBytes); Data temp = new Data<>(key, result, this.context.getDataTime(), this.context.getHeader()); Data convert = super.convert(temp); this.context.forward(convert); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AddTagSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import java.util.Properties; import java.util.function.Supplier; public class AddTagSupplier implements Supplier> { private final Properties properties = new Properties(); public AddTagSupplier(Supplier value) { properties.put(Constant.STREAM_TAG, value.get()); } public AddTagSupplier() { } @Override public Processor get() { return new AddTagProcessor(properties); } class AddTagProcessor extends AbstractProcessor { private final Properties properties = new Properties(); public AddTagProcessor(Properties properties) { this.properties.putAll(properties); } @Override public void process(T data) throws Throwable { Properties header = this.context.getHeader(); header.putAll(properties); Data result = new Data<>(this.context.getKey(), data, this.context.getDataTime(), header); this.context.forward(result); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AggregateSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable; import org.apache.rocketmq.streams.core.function.AggregateAction; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.state.StateStore; import org.apache.rocketmq.streams.core.util.ColumnFamilyUtil; import java.util.function.Supplier; public class AggregateSupplier implements Supplier> { private final String currentName; private final String parentName; private Supplier initAction; private AggregateAction aggregateAction; public AggregateSupplier(String currentName, String parentName, Supplier initAction, AggregateAction aggregateAction) { this.currentName = currentName; this.parentName = parentName; this.initAction = initAction; this.aggregateAction = aggregateAction; } @Override public Processor get() { return new AggregateProcessor(currentName, parentName, initAction, aggregateAction); } private class AggregateProcessor extends AbstractProcessor { private final String currentName; private final String parentName; private final Supplier initAction; private final AggregateAction aggregateAction; private StateStore stateStore; private MessageQueue stateTopicMessageQueue; public AggregateProcessor(String currentName, String parentName, Supplier initAction, AggregateAction aggregateAction) { this.currentName = currentName; this.parentName = parentName; this.initAction = initAction; this.aggregateAction = aggregateAction; } @Override public void preProcess(StreamContext context) throws RecoverStateStoreThrowable { super.preProcess(context); this.stateStore = super.waitStateReplay(); String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX; this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId()); } @Override public void process(V data) throws Throwable { K key = this.context.getKey(); OV value; byte[] keyBytes = super.object2Byte(key); byte[] valueBytes = stateStore.get(ColumnFamilyUtil.VALUE_STATE_CF, keyBytes); if (valueBytes == null || valueBytes.length == 0) { value = initAction.get(); } else { value = super.byte2Object(valueBytes); } OV result = aggregateAction.calculate(key, data, value); byte[] newValueBytes = super.object2Byte(result); stateStore.put(this.stateTopicMessageQueue, ColumnFamilyUtil.VALUE_STATE_CF, keyBytes, newValueBytes); Data temp = new Data<>(key, result, this.context.getDataTime(), this.context.getHeader()); Data convert = super.convert(temp); this.context.forward(convert); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/FilterSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.streams.core.function.FilterAction; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import java.util.function.Supplier; public class FilterSupplier implements Supplier> { private FilterAction filterAction; public FilterSupplier(FilterAction filterAction) { this.filterAction = filterAction; } @Override public Processor get() { return new FilterProcessor(filterAction); } private class FilterProcessor extends AbstractProcessor { private final FilterAction filterAction; public FilterProcessor(FilterAction filterAction) { this.filterAction = filterAction; } @Override public void process(T data) throws Throwable { boolean pass = filterAction.apply(data); if (pass) { Data result = new Data<>(this.context.getKey(), data, this.context.getDataTime(), this.context.getHeader()); this.context.forward(result); } } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/ForeachSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.streams.core.function.ForeachAction; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import java.util.function.Supplier; public class ForeachSupplier implements Supplier> { private ForeachAction foreachAction; public ForeachSupplier(ForeachAction foreachAction) { this.foreachAction = foreachAction; } @Override public Processor get() { return new ForeachProcessor(this.foreachAction); } class ForeachProcessor extends AbstractProcessor { private ForeachAction foreachAction; public ForeachProcessor(ForeachAction foreachAction) { this.foreachAction = foreachAction; } @Override public void process(T data) throws Throwable { this.foreachAction.apply(data); Data result = new Data<>(this.context.getKey(), data, this.context.getDataTime(), this.context.getHeader()); this.context.forward(result); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/JoinAggregateSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable; import org.apache.rocketmq.streams.core.function.ValueJoinAction; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.util.ColumnFamilyUtil; import org.apache.rocketmq.streams.core.window.JoinType; import org.apache.rocketmq.streams.core.window.StreamType; import org.apache.rocketmq.streams.core.state.StateStore; import org.apache.rocketmq.streams.core.util.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Properties; import java.util.function.Supplier; public class JoinAggregateSupplier implements Supplier> { private static final Logger logger = LoggerFactory.getLogger(JoinAggregateSupplier.class.getName()); private String name; private JoinType joinType; private final ValueJoinAction joinAction; public JoinAggregateSupplier(String name, JoinType joinType, ValueJoinAction joinAction) { this.name = name; this.joinType = joinType; this.joinAction = joinAction; } @Override public Processor get() { return new JoinStreamAggregateProcessor(name, joinType, joinAction); } private class JoinStreamAggregateProcessor extends AbstractProcessor { private String name; private JoinType joinType; private final ValueJoinAction joinAction; private MessageQueue stateTopicMessageQueue; private StateStore stateStore; public JoinStreamAggregateProcessor(String name, JoinType joinType, ValueJoinAction joinAction) { this.name = name; this.joinType = joinType; this.joinAction = joinAction; } @Override public void preProcess(StreamContext context) throws RecoverStateStoreThrowable { super.preProcess(context); this.stateStore = super.waitStateReplay(); String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX; this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId()); } @Override public void process(Object data) throws Throwable { Object key = this.context.getKey(); Properties header = this.context.getHeader(); StreamType streamType = (StreamType) header.get(Constant.STREAM_TAG); store(key, data, streamType); fire(key, data, streamType); } private void store(Object key, Object data, StreamType streamType) throws Throwable { String name = Utils.buildKey(this.name, streamType.name()); switch (streamType) { case LEFT_STREAM: case RIGHT_STREAM: { String storeKey = Utils.buildKey(name, super.toHexString(key)); byte[] keyBytes = Utils.object2Byte(storeKey); byte[] valueBytes = super.object2Byte(data); this.stateStore.put(stateTopicMessageQueue, ColumnFamilyUtil.VALUE_STATE_CF, keyBytes, valueBytes); break; } } } @SuppressWarnings("unchecked") private void fire(Object key, Object data, StreamType streamType) throws Throwable { switch (streamType) { case LEFT_STREAM: { String name = Utils.buildKey(this.name, StreamType.RIGHT_STREAM.name()); String storeKey = Utils.buildKey(name, super.toHexString(key)); byte[] keyBytes = Utils.object2Byte(storeKey); byte[] bytes = this.stateStore.get(ColumnFamilyUtil.VALUE_STATE_CF, keyBytes); if (joinType == JoinType.INNER_JOIN) { if (bytes == null || bytes.length == 0) { break; } } else if (joinType == JoinType.LEFT_JOIN) { //no-op } else { throw new UnsupportedOperationException("unknown joinType = " + joinType); } V1 v1Data = (V1) data; V2 v2Data = super.byte2Object(bytes); doFire(v1Data, v2Data); break; } case RIGHT_STREAM: { if (joinType != JoinType.INNER_JOIN) { break; } String name = Utils.buildKey(this.name, StreamType.LEFT_STREAM.name()); String storeKey = Utils.buildKey(name, super.toHexString(key)); byte[] keyBytes = Utils.object2Byte(storeKey); byte[] bytes = this.stateStore.get(ColumnFamilyUtil.VALUE_STATE_CF, keyBytes); if (bytes == null || bytes.length == 0) { break; } V2 v2Data = (V2) data; V1 v1Data = super.byte2Object(bytes); doFire(v1Data, v2Data); break; } } //todo 是否需要删除状态? } private void doFire(V1 v1Data, V2 v2Data) throws Throwable { OUT out = this.joinAction.apply(v1Data, v2Data); Data result = new Data<>(this.context.getKey(), out, this.context.getDataTime(), this.context.getHeader()); Data convert = super.convert(result); this.context.forward(convert); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/JoinWindowAggregateSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RStreamsException; import org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable; import org.apache.rocketmq.streams.core.function.ValueJoinAction; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractWindowProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.window.JoinType; import org.apache.rocketmq.streams.core.window.StreamType; import org.apache.rocketmq.streams.core.window.Window; import org.apache.rocketmq.streams.core.window.WindowInfo; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.window.WindowState; import org.apache.rocketmq.streams.core.window.WindowStore; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.streams.core.window.fire.JoinWindowFire; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Comparator; import java.util.List; import java.util.Properties; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; public class JoinWindowAggregateSupplier implements Supplier> { private static final Logger logger = LoggerFactory.getLogger(JoinWindowAggregateSupplier.class.getName()); private String name; private WindowInfo windowInfo; private final ValueJoinAction joinAction; private JoinType joinType; public JoinWindowAggregateSupplier(String name, WindowInfo windowInfo, ValueJoinAction joinAction) { this.name = name; this.windowInfo = windowInfo; this.joinType = windowInfo.getJoinStream().getJoinType(); this.joinAction = joinAction; } @Override public Processor get() { return new JoinStreamWindowAggregateProcessor(name, windowInfo, joinType, joinAction); } @SuppressWarnings("unchecked") private class JoinStreamWindowAggregateProcessor extends AbstractWindowProcessor { private String name; private final WindowInfo windowInfo; private final JoinType joinType; private ValueJoinAction joinAction; private MessageQueue stateTopicMessageQueue; private WindowStore leftWindowStore; private WindowStore rightWindowStore; public JoinStreamWindowAggregateProcessor(String name, WindowInfo windowInfo, JoinType joinType, ValueJoinAction joinAction) { this.name = Utils.buildKey(name, JoinStreamWindowAggregateProcessor.class.getSimpleName()); this.windowInfo = windowInfo; this.joinType = joinType; this.joinAction = joinAction; } @Override public void preProcess(StreamContext context) throws RecoverStateStoreThrowable { super.preProcess(context); leftWindowStore = new WindowStore<>(super.waitStateReplay(), WindowState::byte2WindowState, WindowState::windowState2Byte); rightWindowStore = new WindowStore<>(super.waitStateReplay(), WindowState::byte2WindowState, WindowState::windowState2Byte); this.idleWindowScaner = context.getDefaultWindowScaner(); String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX; this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId()); this.joinWindowFire = new JoinWindowFire<>(joinType, this.stateTopicMessageQueue, context.copy(), joinAction, leftWindowStore, rightWindowStore, this::watermark); } @Override public void process(Object data) throws Throwable { Object key = this.context.getKey(); long time = this.context.getDataTime(); Properties header = this.context.getHeader(); long watermark = this.watermark(time - allowDelay, stateTopicMessageQueue); if (time < watermark) { logger.warn("discard delay data:[{}]. time of data:{}, watermark:{}", data, time, watermark); return; } WindowInfo.JoinStream stream = (WindowInfo.JoinStream) header.get(Constant.STREAM_TAG); StreamType streamType = stream.getStreamType(); if (streamType == null) { String format = String.format("StreamType is empty, data:%s", data); throw new IllegalStateException(format); } store(key, data, time, watermark, streamType); List fire = this.joinWindowFire.fire(this.name, watermark, streamType); for (WindowKey windowKey : fire) { this.idleWindowScaner.removeWindowKey(windowKey); } } private void store(Object key, Object data, long time, long watermark, StreamType streamType) throws Throwable { String name = Utils.buildKey(this.name, streamType.name()); List windows = super.calculateWindow(windowInfo, time); for (Window window : windows) { logger.debug("timestamp=" + time + ". time -> window: " + Utils.format(time) + "->" + window); WindowKey windowKey = new WindowKey(name, super.toHexString(key), window.getEndTime(), window.getStartTime()); switch (streamType) { case LEFT_STREAM: WindowState leftState = new WindowState<>((K) key, (V1) data, time); this.leftWindowStore.put(stateTopicMessageQueue, windowKey, leftState); this.idleWindowScaner.putJoinWindowCallback(windowKey, watermark, joinWindowFire); break; case RIGHT_STREAM: WindowState rightState = new WindowState<>((K) key, (V2) data, time); this.rightWindowStore.put(stateTopicMessageQueue, windowKey, rightState); this.idleWindowScaner.putJoinWindowCallback(windowKey, watermark, joinWindowFire); break; } } } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/KeySelectSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.streams.core.function.SelectAction; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import java.util.function.Supplier; public class KeySelectSupplier implements Supplier> { private final SelectAction selectAction; public KeySelectSupplier(SelectAction selectAction) { this.selectAction = selectAction; } @Override public Processor get() { return new MapperProcessor(selectAction); } private class MapperProcessor extends AbstractProcessor { private final SelectAction selectAction; public MapperProcessor(SelectAction selectAction) { this.selectAction = selectAction; } @Override public void process(T data) throws Throwable { KEY newKey = selectAction.select(data); Data result = new Data<>(newKey, data, this.context.getDataTime(), this.context.getHeader()); this.context.forward(result); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/MultiValueChangeSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.function.Supplier; public class MultiValueChangeSupplier implements Supplier> { private final ValueMapperAction> valueMapperAction; private static final Logger logger = LoggerFactory.getLogger(MultiValueChangeSupplier.class.getName()); public MultiValueChangeSupplier(ValueMapperAction> valueMapperAction) { this.valueMapperAction = valueMapperAction; } @Override public Processor get() { return new MultiValueMapperProcessor<>(this.valueMapperAction); } static class MultiValueMapperProcessor extends AbstractProcessor { private final ValueMapperAction> valueMapperAction; public MultiValueMapperProcessor(ValueMapperAction> valueMapperAction) { this.valueMapperAction = valueMapperAction; } @Override public void process(T data) throws Throwable { Iterable convert = valueMapperAction.convert(data); if (convert == null) { logger.warn("[{}] converts to null, processor returns directly", data); return; } for (VR item : convert) { if (item == null) { continue; } Data before = new Data<>(this.context.getKey(), item, this.context.getDataTime(), this.context.getHeader()); Data result = convert(before); this.context.forward(result); } } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/PrintSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.util.Utils; import java.util.Properties; import java.util.function.Supplier; public class PrintSupplier implements Supplier> { @Override public Processor get() { return new PrintProcessor<>(); } static class PrintProcessor extends AbstractProcessor { public PrintProcessor() { } @Override public void process(T data) { Properties header = context.getHeader(); Object startTime = header.get(Constant.WINDOW_START_TIME); Object endTime = header.get(Constant.WINDOW_END_TIME); if (startTime == null || endTime == null) { String template = "(key=%s, value=%s)"; Data result = new Data<>(this.context.getKey(), data, this.context.getDataTime(), header); String format = String.format(template, result.getKey(), data.toString()); System.out.println(format); } else { String template = "[%s - %s](key=%s, value=%s)"; String start = Utils.format((Long)startTime); String end = Utils.format((Long)endTime); String format = String.format(template, start, end, this.context.getKey(), data); System.out.println(format); } } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SinkSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.client.producer.DefaultMQProducer; import org.apache.rocketmq.client.producer.selector.SelectMessageQueueByHash; import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.serialization.KeyValueSerializer; import org.apache.rocketmq.streams.core.util.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.function.Supplier; public class SinkSupplier implements Supplier> { private static final Logger logger = LoggerFactory.getLogger(SinkSupplier.class); private final String topicName; private final KeyValueSerializer serializer; public SinkSupplier(String topicName, KeyValueSerializer serializer) { this.topicName = topicName; this.serializer = serializer; } @Override public Processor get() { return new SinkProcessor(this.topicName, this.serializer); } private class SinkProcessor extends AbstractProcessor { private final String topicName; private DefaultMQProducer producer; private final KeyValueSerializer serializer; private K key; public SinkProcessor(String topicName, KeyValueSerializer serializer) { this.topicName = topicName; this.serializer = serializer; } @Override public void preProcess(StreamContext context) { this.context = context; this.producer = context.getDefaultMQProducer(); this.key = context.getKey(); } //sink into shuffle topic/state topic/user topic @Override public void process(T data) throws Throwable { if (data != null) { byte[] value = this.serializer.serialize(key, data); if (value == null || value.length == 0) { //目前RocketMQ不支持发送body为null的消息; return; } Message message; if (this.key == null) { message = new Message(this.topicName, value); message.putUserProperty(Constant.SHUFFLE_VALUE_CLASS_NAME, data.getClass().getName()); if (this.topicName.contains(Constant.SHUFFLE_TOPIC_SUFFIX)) { message.putUserProperty(Constant.SOURCE_TIMESTAMP, String.valueOf(this.context.getDataTime())); } producer.send(message); } else { message = new Message(this.topicName, value); String hexKey = Utils.toHexString(this.key); //the real key is in the body, this key is used to route the same key into the same queue. message.setKeys(hexKey); message.putUserProperty(Constant.SHUFFLE_KEY_CLASS_NAME, this.key.getClass().getName()); message.putUserProperty(Constant.SHUFFLE_VALUE_CLASS_NAME, data.getClass().getName()); if (this.topicName.contains(Constant.SHUFFLE_TOPIC_SUFFIX)) { message.putUserProperty(Constant.SOURCE_TIMESTAMP, String.valueOf(this.context.getDataTime())); } producer.send(message, new SelectMessageQueueByHash(), hexKey); } } } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SourceSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.streams.core.exception.DeserializeThrowable; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.window.TimeType; import org.apache.rocketmq.streams.core.serialization.KeyValueDeserializer; import org.apache.rocketmq.streams.core.util.Pair; import java.util.function.Supplier; public class SourceSupplier implements Supplier> { private String topicName; private KeyValueDeserializer deserializer; public SourceSupplier(String topicName, KeyValueDeserializer deserializer) { this.topicName = topicName; this.deserializer = deserializer; } @Override public Processor get() { return new SourceProcessorImpl(deserializer); } public interface SourceProcessor extends Processor { Pair deserialize(String keyClass, String valueClass, byte[] data) throws DeserializeThrowable; long getTimestamp(MessageExt originData, TimeType timeType); } private class SourceProcessorImpl extends AbstractProcessor implements SourceProcessor { private KeyValueDeserializer deserializer; public SourceProcessorImpl(KeyValueDeserializer deserializer) { this.deserializer = deserializer; } @Override public Pair deserialize(String keyClass, String valueClass, byte[] data) throws DeserializeThrowable { try { this.deserializer.configure(keyClass, valueClass); return this.deserializer.deserialize(data); }catch (Throwable t) { throw new DeserializeThrowable(t); } } @Override public long getTimestamp(MessageExt originData, TimeType timeType) { if (timeType == null) { return System.currentTimeMillis(); } else if (timeType == TimeType.EVENT_TIME) { return originData.getBornTimestamp(); } else if (timeType == TimeType.PROCESS_TIME) { return System.currentTimeMillis(); } else { throw new IllegalStateException("unknown time type: " + timeType.getClass().getName()); } } @Override public void process(V data) throws Throwable { //no-op } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SumAggregate.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.streams.core.function.AggregateAction; import org.apache.rocketmq.streams.core.function.SelectAction; public class SumAggregate implements AggregateAction { private final SelectAction selectAction; public SumAggregate(SelectAction selectAction) { this.selectAction = selectAction; } @Override public Number calculate(K key, V value, Number accumulator) { Number number = selectAction.select(value); if (accumulator == null) { accumulator = number; return accumulator; } if (number instanceof Integer) { return accumulator.intValue() + number.intValue(); } else if (number instanceof Long) { return accumulator.longValue() + number.longValue(); } else if (number instanceof Double) { return accumulator.doubleValue() + number.doubleValue(); } else if (number instanceof Float) { return accumulator.floatValue() + number.floatValue(); } else { throw new UnsupportedOperationException("unsupported number type:" + number.getClass().getName()); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/TimestampSelectorSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.running.StreamContext; import java.util.function.Supplier; public class TimestampSelectorSupplier implements Supplier> { private final ValueMapperAction valueMapperAction; public TimestampSelectorSupplier(ValueMapperAction valueMapperAction) { this.valueMapperAction = valueMapperAction; } @Override public Processor get() { return new TimestampSelector<>(valueMapperAction); } static class TimestampSelector extends AbstractProcessor { private final ValueMapperAction valueMapperAction; public TimestampSelector(ValueMapperAction valueMapperAction) { this.valueMapperAction = valueMapperAction; } @Override public void process(T data) throws Throwable { Long timestamp = this.valueMapperAction.convert(data); StreamContext streamContext = this.context; //override the timestamp of data Data result = new Data<>(streamContext.getKey(), data, timestamp, streamContext.getHeader()); streamContext.forward(result); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/ValueChangeSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.AbstractProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.function.Supplier; public class ValueChangeSupplier implements Supplier> { private final ValueMapperAction valueMapperAction; private static final Logger logger = LoggerFactory.getLogger(ValueChangeSupplier.class.getName()); public ValueChangeSupplier(ValueMapperAction valueMapperAction) { this.valueMapperAction = valueMapperAction; } @Override public Processor get() { return new ValueMapperProcessor<>(this.valueMapperAction); } static class ValueMapperProcessor extends AbstractProcessor { private final ValueMapperAction valueMapperAction; public ValueMapperProcessor(ValueMapperAction valueMapperAction) { this.valueMapperAction = valueMapperAction; } @Override public void process(T data) throws Throwable { O convert = valueMapperAction.convert(data); if (convert == null) { logger.warn("[{}] converts to null, processor returns directly", data); return; } Data before = new Data<>(this.context.getKey(), convert, this.context.getDataTime(), this.context.getHeader()); Data result = convert(before); this.context.forward(result); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/WindowAccumulatorSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable; import org.apache.rocketmq.streams.core.function.SelectAction; import org.apache.rocketmq.streams.core.function.accumulator.Accumulator; import org.apache.rocketmq.streams.core.running.AbstractWindowProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.window.Window; import org.apache.rocketmq.streams.core.window.WindowInfo; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.window.WindowState; import org.apache.rocketmq.streams.core.window.WindowStore; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.streams.core.window.fire.AccumulatorWindowFire; import org.apache.rocketmq.streams.core.window.fire.AccumulatorSessionWindowFire; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; public class WindowAccumulatorSupplier implements Supplier> { private static final Logger logger = LoggerFactory.getLogger(WindowAccumulatorSupplier.class.getName()); private final String name; private WindowInfo windowInfo; private SelectAction selectAction; private Accumulator accumulator; public WindowAccumulatorSupplier(String name, WindowInfo windowInfo, SelectAction selectAction, Accumulator accumulator) { this.name = name; this.windowInfo = windowInfo; this.selectAction = selectAction; this.accumulator = accumulator; } @Override public Processor get() { WindowInfo.WindowType windowType = windowInfo.getWindowType(); switch (windowType) { case SLIDING_WINDOW: case TUMBLING_WINDOW: return new WindowAccumulatorProcessor(name, windowInfo, selectAction, accumulator); case SESSION_WINDOW: return new SessionWindowAccumulatorProcessor(name, windowInfo, selectAction, accumulator); default: throw new RuntimeException("window type is error, WindowType=" + windowType); } } public class WindowAccumulatorProcessor extends AbstractWindowProcessor { private final WindowInfo windowInfo; private String name; private MessageQueue stateTopicMessageQueue; private SelectAction selectAction; private Accumulator accumulator; private WindowStore> windowStore; private final AtomicReference errorReference = new AtomicReference<>(null); public WindowAccumulatorProcessor(String name, WindowInfo windowInfo, SelectAction selectAction, Accumulator accumulator) { this.name = String.join(Constant.SPLIT, name, WindowAccumulatorProcessor.class.getSimpleName()); this.windowInfo = windowInfo; this.selectAction = selectAction; this.accumulator = accumulator; } @Override public void preProcess(StreamContext context) throws RecoverStateStoreThrowable { super.preProcess(context); this.windowStore = new WindowStore<>(super.waitStateReplay(), WindowState::byte2WindowState, WindowState::windowState2Byte); this.idleWindowScaner = context.getDefaultWindowScaner(); String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX; this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId()); this.accumulatorWindowFire = new AccumulatorWindowFire<>(this.windowStore, context.copy(), this.stateTopicMessageQueue, this::watermark); } /** * 维持一个watermark,小于watermark的数据都已经达到,触发窗口计算 */ @Override public void process(V data) throws Throwable { Throwable throwable = errorReference.get(); if (throwable != null) { errorReference.set(null); throw throwable; } K key = this.context.getKey(); long time = this.context.getDataTime(); long watermark = this.watermark(time - allowDelay, stateTopicMessageQueue); if (time < watermark) { //delay data. logger.warn("discard delay data:[{}]. time of data:{}, watermark:{}", data, time, watermark); return; } //f(time) -> List List windows = super.calculateWindow(windowInfo, time); for (Window window : windows) { logger.debug("timestamp=" + time + ". time -> window: " + Utils.format(time) + "->" + window); //f(Window + key, store) -> oldValue //todo key 怎么转化成对应的string,只和key的值有关系 WindowKey windowKey = new WindowKey(name, super.toHexString(key), window.getEndTime(), window.getStartTime()); WindowState> oldState = this.windowStore.get(windowKey); //f(oldValue, Agg) -> newValue Accumulator storeAccumulator; if (oldState == null || oldState.getValue() == null) { storeAccumulator = accumulator.clone(); } else { storeAccumulator = oldState.getValue(); } R select = selectAction.select(data); storeAccumulator.addValue(select); //f(Window + key, newValue, store) WindowState> state = new WindowState<>(key, storeAccumulator, time); this.windowStore.put(stateTopicMessageQueue, windowKey, state); this.idleWindowScaner.putAccumulatorWindowCallback(windowKey, watermark, this.accumulatorWindowFire); } try { List fire = this.accumulatorWindowFire.fire(name, watermark); for (WindowKey windowKey : fire) { this.idleWindowScaner.removeWindowKey(windowKey); } } catch (Throwable t) { errorReference.compareAndSet(null, t); } } } private class SessionWindowAccumulatorProcessor extends AbstractWindowProcessor { private final String name; private final WindowInfo windowInfo; private MessageQueue stateTopicMessageQueue; private SelectAction selectAction; private Accumulator accumulator; private WindowStore> windowStore; public SessionWindowAccumulatorProcessor(String name, WindowInfo windowInfo, SelectAction selectAction, Accumulator accumulator) { this.name = String.join(Constant.SPLIT, name, SessionWindowAccumulatorProcessor.class.getSimpleName()); this.windowInfo = windowInfo; this.selectAction = selectAction; this.accumulator = accumulator; } @Override public void preProcess(StreamContext context) throws RecoverStateStoreThrowable { super.preProcess(context); this.windowStore = new WindowStore<>(super.waitStateReplay(), WindowState::byte2WindowState, WindowState::windowState2Byte); this.idleWindowScaner = context.getDefaultWindowScaner(); String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX; this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId()); this.accumulatorSessionWindowFire = new AccumulatorSessionWindowFire<>(this.windowStore, context.copy(), this.stateTopicMessageQueue, this::watermark); } @Override public void process(V data) throws Throwable { K key = this.context.getKey(); long time = this.context.getDataTime(); long watermark = this.watermark(time - allowDelay, stateTopicMessageQueue); if (time < watermark) { logger.warn("discard delay data:[{}]. time of data:{}, watermark:{}", data, time, watermark); return; } //本地存储里面搜索下 Pair newSessionWindowTime = fireIfSessionOut(key, data, time, watermark); if (newSessionWindowTime != null) { Accumulator temp = accumulator.clone(); R select = selectAction.select(data); temp.addValue(select); WindowState> state = new WindowState<>(key, temp, time); if (time < state.getRecordEarliestTimestamp()) { //更新最早时间戳,用于状态触发时候,作为session 窗口的begin时间戳 state.setRecordEarliestTimestamp(time); } WindowKey windowKey = new WindowKey(name, super.toHexString(key), newSessionWindowTime.getValue(), newSessionWindowTime.getKey()); logger.info("new session window, with key={}, valueTime={}, sessionBegin=[{}], sessionEnd=[{}]", key, Utils.format(time), Utils.format(newSessionWindowTime.getKey()), Utils.format(newSessionWindowTime.getValue())); this.windowStore.put(stateTopicMessageQueue, windowKey, state); this.idleWindowScaner.putAccumulatorSessionWindowCallback(windowKey, watermark, this.accumulatorSessionWindowFire); } } //使用前缀查询找到session state, 触发已经session out的 watermark @SuppressWarnings("unchecked") private Pair fireIfSessionOut(K key, V data, long dataTime, long watermark) throws Throwable { List>>> pairs = this.windowStore.searchMatchKeyPrefix(name); if (pairs.size() == 0) { return new Pair<>(dataTime, dataTime + windowInfo.getSessionTimeout().toMilliseconds()); } logger.debug("exist session state num={}", pairs.size()); //sessionEndTime小的先触发 Iterator>>> iterator = pairs.iterator(); int count = 0; long lastStateSessionEnd = 0; long maxFireSessionEnd = Long.MIN_VALUE; while (iterator.hasNext()) { Pair>> pair = iterator.next(); logger.debug("exist session state{}=[{}]", count++, pair); WindowKey windowKey = pair.getKey(); long sessionEnd = windowKey.getWindowEnd(); if (count == pairs.size()) { lastStateSessionEnd = sessionEnd; } //先触发一遍,触发后从集合中删除 if (sessionEnd < watermark) { //触发state List fire = this.accumulatorSessionWindowFire.fire(name, watermark); for (WindowKey delete : fire) { this.idleWindowScaner.removeWindowKey(delete); } iterator.remove(); maxFireSessionEnd = Long.max(sessionEnd, maxFireSessionEnd); } } if (dataTime < maxFireSessionEnd) { logger.warn("late data, discard. key=[{}], data=[{}], dataTime < maxFireSessionEnd: [{}] < [{}]", key, data, dataTime, maxFireSessionEnd); return null; } boolean createNewSessionWindow = false; WindowKey needToDelete = null; //再次遍历,找到数据属于某个窗口,如果窗口已经关闭,则只计算新的值,如果窗口没有关闭则计算新值、更新窗口边界、存储状态、删除老值 for (int i = 0; i < pairs.size(); i++) { Pair>> pair = pairs.get(i); WindowKey windowKey = pair.getKey(); WindowState> state = pair.getValue(); if (windowKey.getWindowEnd() < dataTime) { createNewSessionWindow = true; } else if (windowKey.getWindowStart() <= dataTime) { logger.debug("data belong to exist session window.dataTime=[{}], window:[{} - {}]", dataTime, Utils.format(windowKey.getWindowStart()), Utils.format(windowKey.getWindowEnd())); Accumulator value = state.getValue(); R select = selectAction.select(data); value.addValue(select); //更新state state.setValue(value); state.setRecordLastTimestamp(dataTime); if (dataTime < state.getRecordEarliestTimestamp()) { //更新最早时间戳,用于状态触发时候,作为session 窗口的begin时间戳 state.setRecordEarliestTimestamp(dataTime); } //如果是最后一个窗口,更新窗口结束时间 if (i == pairs.size() - 1) { long mayBeSessionEnd = dataTime + windowInfo.getSessionTimeout().toMilliseconds(); if (windowKey.getWindowEnd() < mayBeSessionEnd) { logger.debug("update exist session window, before:[{} - {}], after:[{} - {}]", Utils.format(windowKey.getWindowStart()), Utils.format(windowKey.getWindowEnd()), Utils.format(windowKey.getWindowStart()), Utils.format(mayBeSessionEnd)); //删除老状态 needToDelete = windowKey; //需要保存的新状态 windowKey = new WindowKey(windowKey.getOperatorName(), windowKey.getKey2String(), mayBeSessionEnd, windowKey.getWindowStart()); } } } else { logger.warn("discard data: key=[{}], data=[{}], dataTime=[{}], watermark=[{}]", key, data, dataTime, watermark); } this.windowStore.put(stateTopicMessageQueue, windowKey, state); this.idleWindowScaner.putAccumulatorSessionWindowCallback(windowKey, watermark, this.accumulatorSessionWindowFire); this.idleWindowScaner.removeOldAccumulatorSession(needToDelete); this.windowStore.deleteByKey(needToDelete); } if (pairs.size() == 0 || createNewSessionWindow) { return new Pair<>(lastStateSessionEnd, dataTime + windowInfo.getSessionTimeout().toMilliseconds()); } return null; } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/WindowAggregateSupplier.java ================================================ /* * 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.core.function.supplier; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable; import org.apache.rocketmq.streams.core.function.AggregateAction; import org.apache.rocketmq.streams.core.running.AbstractWindowProcessor; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.streams.core.window.Window; import org.apache.rocketmq.streams.core.window.WindowInfo; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.window.WindowState; import org.apache.rocketmq.streams.core.window.WindowStore; import org.apache.rocketmq.streams.core.window.fire.AggregateSessionWindowFire; import org.apache.rocketmq.streams.core.window.fire.AggregateWindowFire; import org.checkerframework.checker.units.qual.K; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; public class WindowAggregateSupplier implements Supplier> { private static final Logger logger = LoggerFactory.getLogger(WindowAggregateSupplier.class.getName()); private String name; private WindowInfo windowInfo; private Supplier initAction; private AggregateAction aggregateAction; public WindowAggregateSupplier(String name, WindowInfo windowInfo, Supplier initAction, AggregateAction aggregateAction) { this.name = name; this.windowInfo = windowInfo; this.initAction = initAction; this.aggregateAction = aggregateAction; } @Override public Processor get() { WindowInfo.WindowType windowType = windowInfo.getWindowType(); switch (windowType) { case SLIDING_WINDOW: case TUMBLING_WINDOW: return new WindowAggregateProcessor(name, windowInfo, initAction, aggregateAction); case SESSION_WINDOW: return new SessionWindowAggregateProcessor(name, windowInfo, initAction, aggregateAction); default: throw new RuntimeException("window type is error, WindowType=" + windowType); } } private class WindowAggregateProcessor extends AbstractWindowProcessor { private final WindowInfo windowInfo; private String name; private Supplier initAction; private AggregateAction aggregateAction; private MessageQueue stateTopicMessageQueue; private WindowStore windowStore; private final AtomicReference errorReference = new AtomicReference<>(null); public WindowAggregateProcessor(String name, WindowInfo windowInfo, Supplier initAction, AggregateAction aggregateAction) { this.name = String.join(Constant.SPLIT, name, WindowAggregateProcessor.class.getSimpleName()); this.windowInfo = windowInfo; this.initAction = initAction; this.aggregateAction = aggregateAction; } @Override public void preProcess(StreamContext context) throws RecoverStateStoreThrowable { super.preProcess(context); this.windowStore = new WindowStore<>(super.waitStateReplay(), WindowState::byte2WindowState, WindowState::windowState2Byte); this.idleWindowScaner = context.getDefaultWindowScaner(); String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX; this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId()); this.aggregateWindowFire = new AggregateWindowFire<>(this.windowStore, this.stateTopicMessageQueue, context.copy(), this::watermark); } /** * 维持一个watermark,小于watermark的数据都已经达到,触发窗口计算 */ @Override public void process(V data) throws Throwable { Throwable throwable = errorReference.get(); if (throwable != null) { errorReference.set(null); throw throwable; } K key = this.context.getKey(); long time = this.context.getDataTime(); long watermark = this.watermark(time - allowDelay, stateTopicMessageQueue); if (time < watermark) { logger.warn("discard delay data:[{}]. time of data:{}, watermark:{}", data, time, watermark); return; } //f(time) -> List List windows = super.calculateWindow(windowInfo, time); for (Window window : windows) { logger.debug("timestamp=" + time + ". time -> window: " + Utils.format(time) + "->" + window); //f(Window + key, store) -> oldValue //todo key 怎么转化成对应的string,只和key的值有关系 WindowKey windowKey = new WindowKey(name, super.toHexString(key), window.getEndTime(), window.getStartTime()); WindowState oldState = this.windowStore.get(windowKey); //f(oldValue, Agg) -> newValue OV oldValue; if (oldState == null || oldState.getValue() == null) { oldValue = initAction.get(); } else { oldValue = oldState.getValue(); } OV newValue = this.aggregateAction.calculate(key, data, oldValue); if (newValue != null && newValue.equals(oldValue)) { continue; } //f(Window + key, newValue, store) WindowState state = new WindowState<>(key, newValue, time); this.windowStore.put(stateTopicMessageQueue, windowKey, state); this.idleWindowScaner.putAggregateWindowCallback(windowKey, watermark, this.aggregateWindowFire); } try { List fire = this.aggregateWindowFire.fire(name, watermark); for (WindowKey windowKey : fire) { this.idleWindowScaner.removeWindowKey(windowKey); } } catch (Throwable t) { errorReference.compareAndSet(null, t); } } } private class SessionWindowAggregateProcessor extends AbstractWindowProcessor { private final String name; private final WindowInfo windowInfo; private Supplier initAction; private AggregateAction aggregateAction; private MessageQueue stateTopicMessageQueue; private WindowStore windowStore; public SessionWindowAggregateProcessor(String name, WindowInfo windowInfo, Supplier initAction, AggregateAction aggregateAction) { this.name = String.join(Constant.SPLIT, name, SessionWindowAggregateProcessor.class.getSimpleName()); this.windowInfo = windowInfo; this.initAction = initAction; this.aggregateAction = aggregateAction; } @Override public void preProcess(StreamContext context) throws RecoverStateStoreThrowable { super.preProcess(context); this.windowStore = new WindowStore<>(super.waitStateReplay(), WindowState::byte2WindowState, WindowState::windowState2Byte); this.idleWindowScaner = context.getDefaultWindowScaner(); String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX; this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId()); this.aggregateSessionWindowFire = new AggregateSessionWindowFire<>(this.windowStore, this.stateTopicMessageQueue, context.copy(), this::watermark); } @Override public void process(V data) throws Throwable { K key = this.context.getKey(); long time = this.context.getDataTime(); long watermark = this.watermark(time - allowDelay, stateTopicMessageQueue); if (time < watermark) { logger.warn("discard delay data:[{}]. time of data:{}, watermark:{}", data, time, watermark); return; } //本地存储里面搜索下 Pair newSessionWindowTime = fireIfSessionOut(key, data, time, watermark); if (newSessionWindowTime != null) { OV oldValue = this.initAction.get(); OV newValue = this.aggregateAction.calculate(key, data, oldValue); WindowState state = new WindowState<>(key, newValue, time); if (time < state.getRecordEarliestTimestamp()) { //更新最早时间戳,用于状态触发时候,作为session 窗口的begin时间戳 state.setRecordEarliestTimestamp(time); } WindowKey windowKey = new WindowKey(name, super.toHexString(key), newSessionWindowTime.getValue(), newSessionWindowTime.getKey()); logger.info("new session window, with key={}, valueTime={}, sessionBegin=[{}], sessionEnd=[{}]", key, Utils.format(time), Utils.format(newSessionWindowTime.getKey()), Utils.format(newSessionWindowTime.getValue())); this.windowStore.put(stateTopicMessageQueue, windowKey, state); this.idleWindowScaner.putAggregateSessionWindowCallback(windowKey, watermark, this.aggregateSessionWindowFire); } } //使用前缀查询找到session state, 触发已经session out的 watermark @SuppressWarnings("unchecked") private Pair fireIfSessionOut(K key, V data, long dataTime, long watermark) throws Throwable { List>> pairs = this.windowStore.searchMatchKeyPrefix(name); if (pairs.size() == 0) { return new Pair<>(dataTime, dataTime + windowInfo.getSessionTimeout().toMilliseconds()); } logger.debug("exist session state num={}", pairs.size()); //sessionEndTime小的先触发 Iterator>> iterator = pairs.iterator(); int count = 0; long lastStateSessionEnd = 0; long maxFireSessionEnd = Long.MIN_VALUE; while (iterator.hasNext()) { Pair> pair = iterator.next(); logger.debug("exist session state{}=[{}]", count++, pair); WindowKey windowKey = pair.getKey(); long sessionEnd = windowKey.getWindowEnd(); if (count == pairs.size()) { lastStateSessionEnd = sessionEnd; } //先触发一遍,触发后从集合中删除 if (sessionEnd < watermark) { //触发state List fire = this.aggregateSessionWindowFire.fire(name, watermark); for (WindowKey delete : fire) { this.idleWindowScaner.removeWindowKey(delete); } iterator.remove(); maxFireSessionEnd = Long.max(sessionEnd, maxFireSessionEnd); } } if (dataTime < maxFireSessionEnd) { logger.warn("late data, discard. key=[{}], data=[{}], dataTime < maxFireSessionEnd: [{}] < [{}]", key, data, dataTime, maxFireSessionEnd); return null; } boolean createNewSessionWindow = false; WindowKey needToDelete = null; //再次遍历,找到数据属于某个窗口,如果窗口已经关闭,则只计算新的值,如果窗口没有关闭则计算新值、更新窗口边界、存储状态、删除老值 for (int i = 0; i < pairs.size(); i++) { Pair> pair = pairs.get(i); WindowKey windowKey = pair.getKey(); WindowState state = pair.getValue(); if (windowKey.getWindowEnd() < dataTime) { createNewSessionWindow = true; } else if (windowKey.getWindowStart() <= dataTime) { logger.debug("data belong to exist session window.dataTime=[{}], window:[{} - {}]", dataTime, Utils.format(windowKey.getWindowStart()), Utils.format(windowKey.getWindowEnd())); OV newValue = this.aggregateAction.calculate(key, data, state.getValue()); //更新state state.setValue(newValue); state.setRecordLastTimestamp(dataTime); if (dataTime < state.getRecordEarliestTimestamp()) { //更新最早时间戳,用于状态触发时候,作为session 窗口的begin时间戳 state.setRecordEarliestTimestamp(dataTime); } //如果是最后一个窗口,更新窗口结束时间 if (i == pairs.size() - 1) { long mayBeSessionEnd = dataTime + windowInfo.getSessionTimeout().toMilliseconds(); if (windowKey.getWindowEnd() < mayBeSessionEnd) { logger.debug("update exist session window, before:[{} - {}], after:[{} - {}]", Utils.format(windowKey.getWindowStart()), Utils.format(windowKey.getWindowEnd()), Utils.format(windowKey.getWindowStart()), Utils.format(mayBeSessionEnd)); //删除老状态 needToDelete = windowKey; //需要保存的新状态 windowKey = new WindowKey(windowKey.getOperatorName(), windowKey.getKey2String(), mayBeSessionEnd, windowKey.getWindowStart()); } } } else { logger.warn("discard data: key=[{}], data=[{}], dataTime=[{}], watermark=[{}]", key, data, dataTime, watermark); } this.windowStore.put(stateTopicMessageQueue, windowKey, state); this.idleWindowScaner.putAggregateSessionWindowCallback(windowKey, watermark, this.aggregateSessionWindowFire); this.idleWindowScaner.removeOldAggregateSession(needToDelete); this.windowStore.deleteByKey(needToDelete); } if (pairs.size() == 0 || createNewSessionWindow) { return new Pair<>(lastStateSessionEnd, dataTime + windowInfo.getSessionTimeout().toMilliseconds()); } return null; } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/metadata/Data.java ================================================ /* * 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.core.metadata; import java.util.Properties; public class Data { private Properties header; private K key; private V value; private Long timestamp; public Data(K key, V value, Long timestamp, Properties header) { this.key = key; this.value = value; this.timestamp = timestamp; this.header = header; } public K getKey() { return key; } public void setKey(K key) { this.key = key; } public V getValue() { return value; } public void setValue(V value) { this.value = value; } public Long getTimestamp() { return timestamp; } public void setTimestamp(Long timestamp) { this.timestamp = timestamp; } public Properties getHeader() { return header; } public void setHeader(Properties header) { this.header = header; } public Data key(NK key) { return new Data<>(key, value, timestamp, new Properties(this.header)); } public Data value(NV value) { return new Data<>(key, value, timestamp, new Properties(this.header)); } @Override public String toString() { return "Data{" + "key=" + key + ", value=" + value + ", timestamp=" + timestamp + '}'; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/metadata/StreamConfig.java ================================================ package org.apache.rocketmq.streams.core.metadata; /* * 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. */ public class StreamConfig { public static final String TIME_TYPE = "timeType"; public static final String ALLOW_LATENESS_MILLISECOND = "allowLatenessMillisecond"; public static final String IDLE_TIME_TO_FIRE_WINDOW = "idleTimeToFireWindow"; public static final String ROCKETMQ_STREAMS_CONSUMER_GROUP = "__source_shuffle_group"; public static final String ROCKETMQ_STREAMS_STATE_CONSUMER_GROUP = "__state_group"; public static final String COMMIT_STATE_INTERNAL_MS = "commitStateIntervalMillisecond"; public static final String ROCKETMQ_STREAMS_CONSUMER_FORM_WHERE = "consume_from_where"; public static Integer STREAMS_PARALLEL_THREAD_NUM = 1; public static Integer SHUFFLE_TOPIC_QUEUE_NUM = 8; public static Integer SCHEDULED_THREAD_NUM = 2; public static Integer DEFAULT_ALLOW_LATE_MILLISECONDS = 2000; } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/GroupedStream.java ================================================ /* * 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.core.rstream; import org.apache.rocketmq.streams.core.function.accumulator.Accumulator; import org.apache.rocketmq.streams.core.function.FilterAction; import org.apache.rocketmq.streams.core.function.SelectAction; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.window.WindowInfo; import org.apache.rocketmq.streams.core.serialization.KeyValueSerializer; import java.util.function.Supplier; public interface GroupedStream { GroupedStream count(); GroupedStream count(SelectAction selectAction); GroupedStream min(SelectAction selectAction); GroupedStream max(SelectAction selectAction); GroupedStream sum(SelectAction selectAction); GroupedStream filter(FilterAction predictor); GroupedStream map(ValueMapperAction valueMapperAction); RStream flatMap(ValueMapperAction> valueMapperAction); GroupedStream aggregate(Accumulator accumulator); WindowStream window(WindowInfo windowInfo); GroupedStream addGraphNode(String name, Supplier> supplier); RStream toRStream(); void sink(String topicName, KeyValueSerializer serializer); } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/GroupedStreamImpl.java ================================================ package org.apache.rocketmq.streams.core.rstream; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.function.AggregateAction; import org.apache.rocketmq.streams.core.function.FilterAction; import org.apache.rocketmq.streams.core.function.SelectAction; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.function.accumulator.Accumulator; import org.apache.rocketmq.streams.core.function.supplier.AccumulatorSupplier; import org.apache.rocketmq.streams.core.function.supplier.AddTagSupplier; import org.apache.rocketmq.streams.core.function.supplier.AggregateSupplier; import org.apache.rocketmq.streams.core.function.supplier.FilterSupplier; import org.apache.rocketmq.streams.core.function.supplier.MultiValueChangeSupplier; import org.apache.rocketmq.streams.core.function.supplier.SinkSupplier; import org.apache.rocketmq.streams.core.function.supplier.SumAggregate; import org.apache.rocketmq.streams.core.function.supplier.ValueChangeSupplier; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.window.WindowInfo; import org.apache.rocketmq.streams.core.serialization.KeyValueSerializer; import org.apache.rocketmq.streams.core.topology.virtual.GraphNode; import org.apache.rocketmq.streams.core.topology.virtual.ProcessorNode; import org.apache.rocketmq.streams.core.topology.virtual.ShuffleProcessorNode; import org.apache.rocketmq.streams.core.topology.virtual.SinkGraphNode; import org.apache.rocketmq.streams.core.util.OperatorNameMaker; import java.util.function.Supplier; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FILTER_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.COUNT_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.ACCUMULATE_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FLAT_MAP_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MAP_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MAX_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MIN_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SINK_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SUM_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.WINDOW_PREFIX; public class GroupedStreamImpl implements GroupedStream { private final Pipeline pipeline; private final GraphNode parent; public GroupedStreamImpl(Pipeline pipeline, GraphNode parent) { this.pipeline = pipeline; this.parent = parent; } @Override public GroupedStream count() { String name = OperatorNameMaker.makeName(COUNT_PREFIX, pipeline.getJobId()); Supplier> supplier = new AggregateSupplier<>(name, parent.getName(), () -> 0, (K key, V value, Integer agg) -> agg + 1); GraphNode graphNode; if (this.parent.shuffleNode()) { graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { graphNode = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent); } @Override public GroupedStream count(SelectAction selectAction) { String name = OperatorNameMaker.makeName(COUNT_PREFIX, pipeline.getJobId()); Supplier> supplier = new AggregateSupplier<>(name, parent.getName(), () -> 0, (K key, V value, Integer agg) -> agg + 1); GraphNode graphNode; if (this.parent.shuffleNode()) { graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { graphNode = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent); } @Override public GroupedStream min(SelectAction selectAction) { String name = OperatorNameMaker.makeName(MIN_PREFIX, pipeline.getJobId()); Supplier> supplier = new AggregateSupplier<>(name, parent.getName(), () -> null, (AggregateAction) (key, value, accumulator) -> { Number number = selectAction.select(value); if (accumulator == null) { return value; } else { Number storedMin = selectAction.select(accumulator); double newValue = number.doubleValue(); double oldValue = storedMin.doubleValue(); if (newValue < oldValue) { return value; } else { return accumulator; } } }); GraphNode graphNode; if (this.parent.shuffleNode()) { graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { graphNode = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent); } @Override public GroupedStream max(SelectAction selectAction) { String name = OperatorNameMaker.makeName(MAX_PREFIX, pipeline.getJobId()); Supplier> supplier = new AggregateSupplier<>(name, parent.getName(), () -> null, (AggregateAction) (key, value, accumulator) -> { Number number = selectAction.select(value); if (accumulator == null) { return value; } else { Number storedMax = selectAction.select(accumulator); double newValue = number.doubleValue(); double oldValue = storedMax.doubleValue(); if (newValue > oldValue) { return value; } else { return accumulator; } } }); GraphNode graphNode; if (this.parent.shuffleNode()) { graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { graphNode = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent); } @Override public GroupedStream sum(SelectAction selectAction) { String name = OperatorNameMaker.makeName(SUM_PREFIX, pipeline.getJobId()); Supplier> supplier = new AggregateSupplier<>(name, parent.getName(), () -> null, new SumAggregate<>(selectAction)); GraphNode graphNode; if (this.parent.shuffleNode()) { graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { graphNode = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent); } @Override public GroupedStream filter(FilterAction predictor) { String name = OperatorNameMaker.makeName(FILTER_PREFIX, pipeline.getJobId()); FilterSupplier supplier = new FilterSupplier<>(predictor); GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier); return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent); } @Override public GroupedStream map(ValueMapperAction mapperAction) { String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId()); ValueChangeSupplier supplier = new ValueChangeSupplier<>(mapperAction); GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier); return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent); } @Override public RStream flatMap(ValueMapperAction> valueMapperAction) { String name = OperatorNameMaker.makeName(FLAT_MAP_PREFIX, pipeline.getJobId()); MultiValueChangeSupplier changeSupplier = new MultiValueChangeSupplier<>(valueMapperAction); GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), changeSupplier); return this.pipeline.addRStreamVirtualNode(graphNode, parent); } @Override public GroupedStream aggregate(Accumulator accumulator) { String name = OperatorNameMaker.makeName(ACCUMULATE_PREFIX, pipeline.getJobId()); Supplier> supplier = new AccumulatorSupplier<>(name, parent.getName(), value -> value, accumulator); GraphNode graphNode; if (this.parent.shuffleNode()) { graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { graphNode = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent); } @Override public WindowStream window(WindowInfo windowInfo) { //需要在window里面shuffle String name = OperatorNameMaker.makeName(WINDOW_PREFIX, pipeline.getJobId()); ProcessorNode node; if (!this.parent.shuffleNode()) { node = new ProcessorNode<>(name, parent.getName(), new AddTagSupplier<>()); } else if (windowInfo.getJoinStream() != null) { node = new ShuffleProcessorNode<>(name, parent.getName(), new AddTagSupplier<>(windowInfo::getJoinStream)); } else { node = new ShuffleProcessorNode<>(name, parent.getName(), new AddTagSupplier<>()); } return this.pipeline.addWindowStreamVirtualNode(node, parent, windowInfo); } @Override public GroupedStream addGraphNode(String name, Supplier> supplier) { GraphNode graphNode; if (this.parent.shuffleNode()) { graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { graphNode = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent); } @Override public RStream toRStream() { return new RStreamImpl<>(this.pipeline, parent); } @Override public void sink(String topicName, KeyValueSerializer serializer) { String name = OperatorNameMaker.makeName(SINK_PREFIX, pipeline.getJobId()); SinkSupplier sinkSupplier = new SinkSupplier<>(topicName, serializer); GraphNode sinkGraphNode = new SinkGraphNode<>(name, parent.getName(), topicName, sinkSupplier); pipeline.addVirtualSink(sinkGraphNode, parent); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/JoinedStream.java ================================================ /* * 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.core.rstream; import org.apache.rocketmq.streams.core.function.SelectAction; import org.apache.rocketmq.streams.core.function.ValueJoinAction; import org.apache.rocketmq.streams.core.function.supplier.AddTagSupplier; import org.apache.rocketmq.streams.core.function.supplier.JoinAggregateSupplier; import org.apache.rocketmq.streams.core.function.supplier.JoinWindowAggregateSupplier; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.window.JoinType; import org.apache.rocketmq.streams.core.window.StreamType; import org.apache.rocketmq.streams.core.window.WindowInfo; import org.apache.rocketmq.streams.core.topology.virtual.GraphNode; import org.apache.rocketmq.streams.core.topology.virtual.ProcessorNode; import org.apache.rocketmq.streams.core.util.OperatorNameMaker; import java.util.ArrayList; import java.util.List; import java.util.Objects; import java.util.function.Supplier; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.ADD_TAG; public class JoinedStream { private RStream leftStream; private RStream rightStream; private JoinType joinType; public JoinedStream(RStream leftStream, RStream rightStream, JoinType joinType) { this.leftStream = leftStream; this.rightStream = rightStream; this.joinType = joinType; } public Where where(SelectAction rightSelectAction) { return new Where<>(rightSelectAction); } public class Where { private SelectAction leftSelectAction; private SelectAction rightSelectAction; public Where(SelectAction leftSelectAction) { this.leftSelectAction = leftSelectAction; } public Where equalTo(SelectAction rightSelectAction) { this.rightSelectAction = rightSelectAction; return this; } public RStream apply(ValueJoinAction joinAction) { List temp = new ArrayList<>(); Pipeline leftStreamPipeline = JoinedStream.this.leftStream.getPipeline(); String jobId = leftStreamPipeline.getJobId(); String name = OperatorNameMaker.makeName(OperatorNameMaker.JOIN_PREFIX, jobId); Supplier> supplier = new JoinAggregateSupplier<>(name, joinType, joinAction); ProcessorNode commChild = new ProcessorNode(name, temp, supplier); { GroupedStream leftGroupedStream = JoinedStream.this.leftStream.keyBy(leftSelectAction); String addTagName = OperatorNameMaker.makeName(ADD_TAG, jobId); leftGroupedStream.addGraphNode(addTagName, new AddTagSupplier<>(() -> StreamType.LEFT_STREAM)); GraphNode lastNode = leftStreamPipeline.getLastNode(); temp.add(lastNode.getName()); commChild.addParent(lastNode); } Pipeline rightStreamPipeline = JoinedStream.this.rightStream.getPipeline(); String rightJobId = rightStreamPipeline.getJobId(); if (!Objects.equals(jobId, rightJobId)) { throw new IllegalStateException("left stream and right stream must have same jobId."); } { GroupedStream rightGroupedStream = JoinedStream.this.rightStream.keyBy(rightSelectAction); String addTagName = OperatorNameMaker.makeName(ADD_TAG, jobId); rightGroupedStream.addGraphNode(addTagName, new AddTagSupplier<>(()-> StreamType.RIGHT_STREAM)); GraphNode lastNode = rightStreamPipeline.getLastNode(); temp.add(lastNode.getName()); commChild.addParent(lastNode); lastNode.addChild(commChild); } return new RStreamImpl<>(leftStreamPipeline, commChild); } public JoinWindow window(WindowInfo windowInfo) { return new JoinWindow<>(this.leftSelectAction, this.rightSelectAction, windowInfo); } } public class JoinWindow { private SelectAction leftSelectAction; private SelectAction rightSelectAction; private WindowInfo windowInfo; public JoinWindow(SelectAction leftSelectAction, SelectAction rightSelectAction, WindowInfo windowInfo) { this.leftSelectAction = leftSelectAction; this.rightSelectAction = rightSelectAction; this.windowInfo = windowInfo; } public RStream apply(ValueJoinAction joinAction) { List temp = new ArrayList<>(); WindowInfo.JoinStream joinStream = new WindowInfo.JoinStream(JoinedStream.this.joinType, null); windowInfo.setJoinStream(joinStream); Pipeline leftStreamPipeline = JoinedStream.this.leftStream.getPipeline(); String jobId = leftStreamPipeline.getJobId(); String name = OperatorNameMaker.makeName(OperatorNameMaker.JOIN_WINDOW_PREFIX, jobId); Supplier> supplier = new JoinWindowAggregateSupplier<>(name, windowInfo, joinAction); ProcessorNode commChild = new ProcessorNode(name, temp, supplier); { GroupedStream leftGroupedStream = JoinedStream.this.leftStream.keyBy(leftSelectAction); WindowInfo leftWindowInfo = this.copy(windowInfo); WindowInfo.JoinStream leftStream = new WindowInfo.JoinStream(JoinedStream.this.joinType, StreamType.LEFT_STREAM); leftWindowInfo.setJoinStream(leftStream); leftGroupedStream.window(leftWindowInfo); GraphNode lastNode = leftStreamPipeline.getLastNode(); temp.add(lastNode.getName()); commChild.addParent(lastNode); } { GroupedStream rightGroupedStream = JoinedStream.this.rightStream.keyBy(rightSelectAction); WindowInfo rightWindowInfo = this.copy(windowInfo); WindowInfo.JoinStream leftStream = new WindowInfo.JoinStream(JoinedStream.this.joinType, StreamType.RIGHT_STREAM); rightWindowInfo.setJoinStream(leftStream); rightGroupedStream.window(rightWindowInfo); Pipeline rightStreamPipeline = JoinedStream.this.rightStream.getPipeline(); String rightJobId = rightStreamPipeline.getJobId(); if (!Objects.equals(jobId, rightJobId)) { throw new IllegalStateException("left stream and right stream must have same jobId."); } GraphNode lastNode = rightStreamPipeline.getLastNode(); temp.add(lastNode.getName()); commChild.addParent(lastNode); lastNode.addChild(commChild); } return new RStreamImpl<>(leftStreamPipeline, commChild); } private WindowInfo copy(WindowInfo windowInfo) { WindowInfo result = new WindowInfo(); WindowInfo.JoinStream joinStream = windowInfo.getJoinStream(); if (joinStream != null) { WindowInfo.JoinStream stream = new WindowInfo.JoinStream(joinStream.getJoinType(), joinStream.getStreamType()); result.setJoinStream(stream); } result.setSessionTimeout(windowInfo.getSessionTimeout()); result.setWindowType(windowInfo.getWindowType()); result.setWindowSize(windowInfo.getWindowSize()); result.setWindowSlide(windowInfo.getWindowSlide()); return result; } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/Pipeline.java ================================================ package org.apache.rocketmq.streams.core.rstream; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.window.WindowInfo; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.topology.virtual.AbstractGraphNode; import org.apache.rocketmq.streams.core.topology.virtual.GraphNode; import java.util.ArrayList; import java.util.List; public class Pipeline { private final String jobId; private final List virtualNodes = new ArrayList<>(); private final GraphNode root = new AbstractGraphNode("root") { @Override public void addRealNode(TopologyBuilder builder) { //no-op } }; public Pipeline(String jobId) { this.jobId = jobId; } public RStream addVirtualSource(GraphNode sourceGraphNode) { root.addChild(sourceGraphNode); virtualNodes.add(sourceGraphNode); return new RStreamImpl<>(this, sourceGraphNode); } public RStream addRStreamVirtualNode(GraphNode currentNode, GraphNode parentNode) { if (!virtualNodes.contains(parentNode)) { virtualNodes.add(parentNode); } parentNode.addChild(currentNode); currentNode.addParent(parentNode); virtualNodes.add(currentNode); return new RStreamImpl<>(this, currentNode); } public GroupedStreamImpl addGroupedStreamVirtualNode(GraphNode currentNode, GraphNode parentNode) { if (!virtualNodes.contains(parentNode)) { virtualNodes.add(parentNode); } parentNode.addChild(currentNode); currentNode.addParent(parentNode); virtualNodes.add(currentNode); return new GroupedStreamImpl<>(this, currentNode); } public WindowStream addWindowStreamVirtualNode(GraphNode currentNode, GraphNode parentNode, WindowInfo windowInfo) { if (!virtualNodes.contains(parentNode)) { virtualNodes.add(parentNode); } parentNode.addChild(currentNode); currentNode.addParent(parentNode); virtualNodes.add(currentNode); return new WindowStreamImpl<>(this, currentNode, windowInfo); } public void addVirtualSink(GraphNode currentNode, GraphNode parentNode) { if (!virtualNodes.contains(parentNode)) { virtualNodes.add(parentNode); } parentNode.addChild(currentNode); virtualNodes.add(currentNode); } public GraphNode getRoot() { return this.root; } public GraphNode getLastNode() { return this.virtualNodes.get(virtualNodes.size() - 1); } public String getJobId() { return jobId; } public int getVirtualNodesNum() { return virtualNodes.size(); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/RStream.java ================================================ /* * 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.core.rstream; import org.apache.rocketmq.streams.core.function.FilterAction; import org.apache.rocketmq.streams.core.function.ForeachAction; import org.apache.rocketmq.streams.core.function.SelectAction; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.serialization.KeyValueSerializer; public interface RStream { RStream selectTimestamp(ValueMapperAction timestampSelector); RStream map(ValueMapperAction mapperAction); RStream flatMap(final ValueMapperAction> mapper); RStream filter(FilterAction predictor); GroupedStream keyBy(SelectAction selectAction); void print(); RStream foreach(ForeachAction foreachAction); JoinedStream join(RStream rightStream); JoinedStream leftJoin(RStream rightStream); Pipeline getPipeline(); void sink(String topicName, KeyValueSerializer serializer); } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/RStreamImpl.java ================================================ /* * 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.core.rstream; import org.apache.rocketmq.streams.core.function.supplier.MultiValueChangeSupplier; import org.apache.rocketmq.streams.core.util.OperatorNameMaker; import org.apache.rocketmq.streams.core.function.FilterAction; import org.apache.rocketmq.streams.core.function.ForeachAction; import org.apache.rocketmq.streams.core.function.SelectAction; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.function.supplier.FilterSupplier; import org.apache.rocketmq.streams.core.function.supplier.ForeachSupplier; import org.apache.rocketmq.streams.core.function.supplier.KeySelectSupplier; import org.apache.rocketmq.streams.core.function.supplier.PrintSupplier; import org.apache.rocketmq.streams.core.function.supplier.SinkSupplier; import org.apache.rocketmq.streams.core.function.supplier.TimestampSelectorSupplier; import org.apache.rocketmq.streams.core.function.supplier.ValueChangeSupplier; import org.apache.rocketmq.streams.core.window.JoinType; import org.apache.rocketmq.streams.core.serialization.KeyValueSerializer; import org.apache.rocketmq.streams.core.topology.virtual.GraphNode; import org.apache.rocketmq.streams.core.topology.virtual.ProcessorNode; import org.apache.rocketmq.streams.core.topology.virtual.SinkGraphNode; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FILTER_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FLAT_MAP_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FOR_EACH_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.GROUPBY_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MAP_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.PRINT_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SINK_PREFIX; public class RStreamImpl implements RStream { private final Pipeline pipeline; private final GraphNode parent; public RStreamImpl(Pipeline pipeline, GraphNode parent) { this.pipeline = pipeline; this.parent = parent; } @Override public RStream selectTimestamp(ValueMapperAction timestampSelector) { String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId()); TimestampSelectorSupplier supplier = new TimestampSelectorSupplier<>(timestampSelector); GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), supplier); return pipeline.addRStreamVirtualNode(processorNode, parent); } @Override public RStream map(ValueMapperAction mapperAction) { String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId()); ValueChangeSupplier supplier = new ValueChangeSupplier<>(mapperAction); GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), supplier); return pipeline.addRStreamVirtualNode(processorNode, parent); } @Override public RStream flatMap(ValueMapperAction> mapper) { String name = OperatorNameMaker.makeName(FLAT_MAP_PREFIX, pipeline.getJobId()); MultiValueChangeSupplier changeSupplier = new MultiValueChangeSupplier<>(mapper); GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), changeSupplier); return pipeline.addRStreamVirtualNode(processorNode, parent); } @Override public RStream filter(FilterAction predictor) { String name = OperatorNameMaker.makeName(FILTER_PREFIX, pipeline.getJobId()); FilterSupplier supplier = new FilterSupplier<>(predictor); GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), supplier); return pipeline.addRStreamVirtualNode(processorNode, parent); } @Override public GroupedStream keyBy(SelectAction selectAction) { String name = OperatorNameMaker.makeName(GROUPBY_PREFIX, pipeline.getJobId()); KeySelectSupplier keySelectSupplier = new KeySelectSupplier<>(selectAction); GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), true, keySelectSupplier); return pipeline.addGroupedStreamVirtualNode(processorNode, parent); } @Override public void print() { String name = OperatorNameMaker.makeName(PRINT_PREFIX, pipeline.getJobId()); PrintSupplier printSupplier = new PrintSupplier<>(); GraphNode sinkGraphNode = new SinkGraphNode<>(name, parent.getName(), null, printSupplier); pipeline.addVirtualSink(sinkGraphNode, parent); } @Override public RStream foreach(ForeachAction foreachAction) { String name = OperatorNameMaker.makeName(FOR_EACH_PREFIX, pipeline.getJobId()); ForeachSupplier supplier = new ForeachSupplier(foreachAction); ProcessorNode node = new ProcessorNode<>(name, parent.getName(), supplier); return pipeline.addRStreamVirtualNode(node, parent); } @Override public JoinedStream join(RStream rightStream) { return new JoinedStream<>(this, rightStream, JoinType.INNER_JOIN); } @Override public JoinedStream leftJoin(RStream rightStream) { return new JoinedStream<>(this, rightStream, JoinType.LEFT_JOIN); } @Override public Pipeline getPipeline() { return pipeline; } @Override public void sink(String topicName, KeyValueSerializer serializer) { String name = OperatorNameMaker.makeName(SINK_PREFIX, pipeline.getJobId()); SinkSupplier sinkSupplier = new SinkSupplier<>(topicName, serializer); GraphNode sinkGraphNode = new SinkGraphNode<>(name, parent.getName(), topicName, sinkSupplier); pipeline.addVirtualSink(sinkGraphNode, parent); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/StreamBuilder.java ================================================ /* * 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.core.rstream; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.serialization.KeyValueDeserializer; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.topology.virtual.GraphNode; import org.apache.rocketmq.streams.core.topology.virtual.SourceGraphNode; import org.apache.rocketmq.streams.core.util.OperatorNameMaker; import java.util.ArrayList; import java.util.List; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SOURCE_PREFIX; public class StreamBuilder { private final List pipelines = new ArrayList<>(); private final TopologyBuilder topologyBuilder; private final String jobId; public StreamBuilder(String jobId) throws Throwable { checkJobId(jobId); this.jobId = jobId; this.topologyBuilder = new TopologyBuilder(jobId); } private void checkJobId(String jobId) { if (jobId.startsWith(Constant.WATERMARK_KEY)) { throw new IllegalArgumentException("jobId startsWith watermarkPrefix. jodId=" + jobId); } } public RStream source(String topicName, KeyValueDeserializer deserializer) { Pipeline pipeline = new Pipeline(jobId); this.pipelines.add(pipeline); String name = OperatorNameMaker.makeName(SOURCE_PREFIX, jobId); GraphNode sourceGraphNode = new SourceGraphNode<>(name, topicName, deserializer); return pipeline.addVirtualSource(sourceGraphNode); } public TopologyBuilder build() { //双流join场景中,添加共同节点的pipeline最后构建;三流join未验证。 pipelines.sort((o1, o2) -> o2.getVirtualNodesNum() - o1.getVirtualNodesNum()); for (Pipeline pipeline : pipelines) { doBuild(pipeline.getRoot()); } return topologyBuilder; } private void doBuild(GraphNode graphNode) { graphNode.addRealNode(topologyBuilder); List allChild = graphNode.getAllChild(); for (GraphNode node : allChild) { doBuild(node); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/WindowStream.java ================================================ /* * 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.core.rstream; import org.apache.rocketmq.streams.core.function.AggregateAction; import org.apache.rocketmq.streams.core.function.FilterAction; import org.apache.rocketmq.streams.core.function.SelectAction; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.function.accumulator.Accumulator; import org.apache.rocketmq.streams.core.serialization.KeyValueSerializer; /** * WindowStream must be generated from keyBy. * @param key type * @param value type */ public interface WindowStream { WindowStream count(); WindowStream avg(); WindowStream min(SelectAction selectAction); WindowStream max(SelectAction selectAction); WindowStream sum(SelectAction selectAction); WindowStream filter(FilterAction predictor); WindowStream map(ValueMapperAction mapperAction); WindowStream aggregate(AggregateAction aggregateAction); WindowStream aggregate(Accumulator accumulator); RStream toRStream(); void sink(String topicName, KeyValueSerializer serializer); } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/WindowStreamImpl.java ================================================ /* * 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.core.rstream; import org.apache.rocketmq.streams.core.function.AggregateAction; import org.apache.rocketmq.streams.core.function.FilterAction; import org.apache.rocketmq.streams.core.function.SelectAction; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.function.accumulator.Accumulator; import org.apache.rocketmq.streams.core.function.accumulator.AvgAccumulator; import org.apache.rocketmq.streams.core.function.accumulator.CountAccumulator; import org.apache.rocketmq.streams.core.function.supplier.AggregateSupplier; import org.apache.rocketmq.streams.core.function.supplier.FilterSupplier; import org.apache.rocketmq.streams.core.function.supplier.SinkSupplier; import org.apache.rocketmq.streams.core.function.supplier.SumAggregate; import org.apache.rocketmq.streams.core.function.supplier.ValueChangeSupplier; import org.apache.rocketmq.streams.core.function.supplier.WindowAccumulatorSupplier; import org.apache.rocketmq.streams.core.function.supplier.WindowAggregateSupplier; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.serialization.KeyValueSerializer; import org.apache.rocketmq.streams.core.topology.virtual.GraphNode; import org.apache.rocketmq.streams.core.topology.virtual.ProcessorNode; import org.apache.rocketmq.streams.core.topology.virtual.ShuffleProcessorNode; import org.apache.rocketmq.streams.core.topology.virtual.SinkGraphNode; import org.apache.rocketmq.streams.core.util.OperatorNameMaker; import org.apache.rocketmq.streams.core.window.WindowInfo; import java.util.function.Supplier; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.COUNT_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FILTER_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MAP_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MAX_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MIN_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SINK_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SUM_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.WINDOW_AVG_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.AGGREGATE_PREFIX; public class WindowStreamImpl implements WindowStream { private final Pipeline pipeline; private final GraphNode parent; private final WindowInfo windowInfo; public WindowStreamImpl(Pipeline pipeline, GraphNode parent, WindowInfo windowInfo) { this.pipeline = pipeline; this.parent = parent; this.windowInfo = windowInfo; } @Override public WindowStream count() { String name = OperatorNameMaker.makeName(COUNT_PREFIX, pipeline.getJobId()); Supplier> supplier = new WindowAccumulatorSupplier<>(name, windowInfo, value -> value, new CountAccumulator<>()); //是否需要分组计算 ProcessorNode node; if (this.parent.shuffleNode()) { node = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { node = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addWindowStreamVirtualNode(node, parent, windowInfo); } @Override public WindowStream avg() { String name = OperatorNameMaker.makeName(WINDOW_AVG_PREFIX, pipeline.getJobId()); Supplier> supplier = new WindowAccumulatorSupplier<>(name, windowInfo, value -> value, new AvgAccumulator<>()); //是否需要分组计算 ProcessorNode node; if (this.parent.shuffleNode()) { node = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { node = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addWindowStreamVirtualNode(node, parent, windowInfo); } @Override public WindowStream min(SelectAction selectAction) { String name = OperatorNameMaker.makeName(MIN_PREFIX, pipeline.getJobId()); Supplier> supplier = new WindowAggregateSupplier<>(name, windowInfo, () -> null, (AggregateAction) (key, value, accumulator) -> { Number number = selectAction.select(value); if (accumulator == null) { return value; } else { Number storedMin = selectAction.select(accumulator); double newValue = number.doubleValue(); double oldValue = storedMin.doubleValue(); if (newValue < oldValue) { return value; } else { return accumulator; } } }); GraphNode graphNode; if (this.parent.shuffleNode()) { graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { graphNode = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo); } @Override public WindowStream max(SelectAction selectAction) { String name = OperatorNameMaker.makeName(MAX_PREFIX, pipeline.getJobId()); Supplier> supplier = new WindowAggregateSupplier<>(name, windowInfo, () -> null, (AggregateAction) (key, value, accumulator) -> { Number number = selectAction.select(value); if (accumulator == null) { return value; } else { Number storedMax = selectAction.select(accumulator); double newValue = number.doubleValue(); double oldValue = storedMax.doubleValue(); if (newValue > oldValue) { return value; } else { return accumulator; } } }); GraphNode graphNode; if (this.parent.shuffleNode()) { graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { graphNode = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo); } @Override public WindowStream sum(SelectAction selectAction) { String name = OperatorNameMaker.makeName(SUM_PREFIX, pipeline.getJobId()); Supplier> supplier = new WindowAggregateSupplier<>(name, windowInfo, () -> null, new SumAggregate<>(selectAction)); GraphNode graphNode; if (this.parent.shuffleNode()) { graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { graphNode = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo); } @Override public WindowStream filter(FilterAction predictor) { String name = OperatorNameMaker.makeName(FILTER_PREFIX, pipeline.getJobId()); FilterSupplier supplier = new FilterSupplier<>(predictor); GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier); return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo); } @Override public WindowStream map(ValueMapperAction mapperAction) { String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId()); ValueChangeSupplier supplier = new ValueChangeSupplier<>(mapperAction); GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier); return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo); } @Override public WindowStream aggregate(AggregateAction aggregateAction) { String name = OperatorNameMaker.makeName(AGGREGATE_PREFIX, pipeline.getJobId()); Supplier> supplier = new WindowAggregateSupplier<>(name, windowInfo, () -> null, aggregateAction); //是否需要分组计算 ProcessorNode node; if (this.parent.shuffleNode()) { node = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { node = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addWindowStreamVirtualNode(node, parent, windowInfo); } @Override public WindowStream aggregate(Accumulator accumulator) { String name = OperatorNameMaker.makeName(AGGREGATE_PREFIX, pipeline.getJobId()); Supplier> supplier = new WindowAccumulatorSupplier<>(name, windowInfo, value -> value, accumulator); //是否需要分组计算 ProcessorNode node; if (this.parent.shuffleNode()) { node = new ShuffleProcessorNode<>(name, parent.getName(), supplier); } else { node = new ProcessorNode<>(name, parent.getName(), supplier); } return this.pipeline.addWindowStreamVirtualNode(node, parent, windowInfo); } @Override public RStream toRStream() { return new RStreamImpl<>(this.pipeline, parent); } @Override public void sink(String topicName, KeyValueSerializer serializer) { String name = OperatorNameMaker.makeName(SINK_PREFIX, pipeline.getJobId()); SinkSupplier sinkSupplier = new SinkSupplier<>(topicName, serializer); GraphNode sinkGraphNode = new SinkGraphNode<>(name, parent.getName(), topicName, sinkSupplier); pipeline.addVirtualSink(sinkGraphNode, parent); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/AbstractProcessor.java ================================================ package org.apache.rocketmq.streams.core.running; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import com.fasterxml.jackson.core.JsonProcessingException; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.metadata.StreamConfig; import org.apache.rocketmq.streams.core.state.StateStore; import org.apache.rocketmq.streams.core.util.Utils; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.List; public abstract class AbstractProcessor implements Processor { private final List> children = new ArrayList<>(); protected StreamContext context; protected long allowDelay = 0; @Override public void addChild(Processor processor) { children.add(processor); } @Override public void preProcess(StreamContext context) throws RecoverStateStoreThrowable { this.context = context; this.context.init(getChildren()); Object delayObj = this.context .getUserProperties() .getOrDefault(StreamConfig.ALLOW_LATENESS_MILLISECOND, StreamConfig.DEFAULT_ALLOW_LATE_MILLISECONDS); this.allowDelay = Long.parseLong(String.valueOf(delayObj)); } protected List> getChildren() { return Collections.unmodifiableList(children); } protected StateStore waitStateReplay() throws RecoverStateStoreThrowable { MessageQueue sourceTopicQueue = new MessageQueue(context.getSourceTopic(), context.getSourceBrokerName(), context.getSourceQueueId()); StateStore stateStore = context.getStateStore(); stateStore.waitIfNotReady(sourceTopicQueue); return stateStore; } @SuppressWarnings("unchecked") protected Data convert(Data data) { return (Data) new Data<>(data.getKey(), data.getValue(), data.getTimestamp(), data.getHeader()); } private final ByteBuf buf = Unpooled.buffer(16); /** * encode *
     * +-----------+---------------+-------------+-------------+
     * | Int(4)    |   className  | Int(4)       | value bytes |
     * | classname |              |object length |             |
     * +-----------+--------------+---------------+-------------+
     * 
* @param obj the object to serialize; * @return byte[] the result of serialize * @throws JsonProcessingException serialize exception. */ protected byte[] object2Byte(Object obj) throws JsonProcessingException { if (obj == null) { return new byte[]{}; } String name = obj.getClass().getName(); byte[] className = name.getBytes(StandardCharsets.UTF_8); byte[] objBytes = Utils.object2Byte(obj); buf.writeInt(className.length); buf.writeBytes(className); buf.writeInt(objBytes.length); buf.writeBytes(objBytes); byte[] bytes = new byte[buf.readableBytes()]; buf.readBytes(bytes); buf.clear(); return bytes; } /** * decode *
     * +-----------+---------------+-------------+-------------+
     * | Int(4)    |   className  | Int(4)       | value bytes |
     * | classname |              |object length |             |
     * +-----------+--------------+---------------+-------------+
     * 
* @param bytes the byte array to deserialize; * @return V the result of deserialize * @throws Throwable deserialize exception. */ @SuppressWarnings("unchecked") public V byte2Object(byte[] bytes) throws Throwable { if (bytes == null || bytes.length == 0) { return null; } ByteBuf byteBuf = Unpooled.wrappedBuffer(bytes); int classNameLength = byteBuf.readInt(); ByteBuf classNameBuf = byteBuf.readBytes(classNameLength); byte[] clazzNameBytes = new byte[classNameBuf.readableBytes()]; classNameBuf.readBytes(clazzNameBytes); //实例化 String className = new String(clazzNameBytes, StandardCharsets.UTF_8); Class clazz = (Class)Class.forName(className); int objectLength = byteBuf.readInt(); ByteBuf objBuf = byteBuf.readBytes(objectLength); byte[] objectBytes = new byte[objectLength]; objBuf.readBytes(objectBytes); classNameBuf.release(); objBuf.release(); return Utils.byte2Object(objectBytes, clazz); } protected String toHexString(Object source) throws JsonProcessingException { if (source == null) { return null; } if (source instanceof String) { return (String) source; } byte[] sourceByte = this.object2Byte(source); return Utils.toHexString(sourceByte); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/AbstractWindowProcessor.java ================================================ /* * 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.core.running; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RStreamsException; import org.apache.rocketmq.streams.core.state.StateStore; import org.apache.rocketmq.streams.core.util.ColumnFamilyUtil; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.streams.core.window.fire.IdleWindowScaner; import org.apache.rocketmq.streams.core.window.Window; import org.apache.rocketmq.streams.core.window.WindowInfo; import org.apache.rocketmq.streams.core.window.fire.AccumulatorWindowFire; import org.apache.rocketmq.streams.core.window.fire.AccumulatorSessionWindowFire; import org.apache.rocketmq.streams.core.window.fire.AggregateSessionWindowFire; import org.apache.rocketmq.streams.core.window.fire.AggregateWindowFire; import org.apache.rocketmq.streams.core.window.fire.JoinWindowFire; import java.util.ArrayList; import java.util.List; public abstract class AbstractWindowProcessor extends AbstractProcessor { protected IdleWindowScaner idleWindowScaner; protected AccumulatorWindowFire accumulatorWindowFire; protected AccumulatorSessionWindowFire accumulatorSessionWindowFire; protected AggregateWindowFire aggregateWindowFire; protected AggregateSessionWindowFire aggregateSessionWindowFire; protected JoinWindowFire joinWindowFire; protected List calculateWindow(WindowInfo windowInfo, long valueTime) { long sizeInterval = windowInfo.getWindowSize().toMillSecond(); long slideInterval = windowInfo.getWindowSlide().toMillSecond(); List result = new ArrayList<>((int) (sizeInterval / slideInterval)); long lastStart = valueTime - (valueTime + slideInterval) % slideInterval; for (long start = lastStart; start > valueTime - sizeInterval; start -= slideInterval) { long end = start + sizeInterval; Window window = new Window(start, end); result.add(window); } return result; } protected long watermark(long watermark, MessageQueue stateTopicMessageQueue) { byte[] keyBytes = Utils.watermarkKeyBytes(stateTopicMessageQueue, Constant.WATERMARK_KEY); try { StateStore stateStore = this.context.getStateStore(); byte[] watermarkBytes = stateStore.get(ColumnFamilyUtil.WATERMARK_STATE_CF, keyBytes); long oldWatermark = Utils.bytes2Long(watermarkBytes); if (watermark > oldWatermark) { byte[] newWatermarkBytes = Utils.long2Bytes(watermark); stateStore.put(stateTopicMessageQueue, ColumnFamilyUtil.WATERMARK_STATE_CF, keyBytes, newWatermarkBytes); } else { watermark = oldWatermark; } } catch (Throwable t) { throw new RStreamsException(t); } return watermark; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/MessageQueueListenerWrapper.java ================================================ package org.apache.rocketmq.streams.core.running; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.client.consumer.MessageQueueListener; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.HashSet; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.function.BiFunction; import java.util.function.Function; import static org.apache.rocketmq.streams.core.common.Constant.STATE_TOPIC_SUFFIX; class MessageQueueListenerWrapper implements MessageQueueListener { private static final Logger logger = LoggerFactory.getLogger(MessageQueueListenerWrapper.class.getName()); private final MessageQueueListener originListener; private final TopologyBuilder topologyBuilder; private final ConcurrentHashMap> ownedMapping = new ConcurrentHashMap<>(); private final ConcurrentHashMap> mq2Processor = new ConcurrentHashMap<>(); private BiFunction, Set, Throwable> recoverHandler; private Function, Throwable> resetOffsetHandler; MessageQueueListenerWrapper(MessageQueueListener originListener, TopologyBuilder topologyBuilder) { this.originListener = originListener; this.topologyBuilder = topologyBuilder; } @Override public void messageQueueChanged(String topic, Set mqAll, Set mqDivided) { Set ownedQueues = ownedMapping.computeIfAbsent(topic, s -> new HashSet<>()); HashSet addQueue = new HashSet<>(mqDivided); addQueue.removeAll(ownedQueues); HashSet removeQueue = new HashSet<>(ownedQueues); removeQueue.removeAll(mqDivided); ownedQueues.addAll(new HashSet<>(addQueue)); ownedQueues.removeAll(new HashSet<>(removeQueue)); //从shuffle topic中读出的数据才能进行有状态计算。 if (topic.endsWith(Constant.SHUFFLE_TOPIC_SUFFIX)) { Throwable throwable = this.recoverHandler.apply(addQueue, removeQueue); if (throwable != null) { throw new RuntimeException(throwable); } logger.info("recover messageQueue finish, addQueue: [{}], removeQueue:[{}].", addQueue, removeQueue); } buildTask(addQueue); //设计的不太好,移除q,添加消费任务之前,应该加一个状态移除函数;目前这样写的问题是:状态提前移除/加载了,consumer其实仍然在从某个将要移除的q中拉取数据,但是状态却被移除了。 //也不能把originListener.messageQueueChanged放在loadState/removeState之前,那样会已经在拉取数据了,但是状态没有加载好。 originListener.messageQueueChanged(topic, mqAll, mqDivided); removeTask(removeQueue); } private void buildTask(Set addQueues) { for (MessageQueue messageQueue : addQueues) { String key = Utils.buildKey(messageQueue.getBrokerName(), messageQueue.getTopic(), messageQueue.getQueueId()); if (!mq2Processor.containsKey(key)) { Processor processor = topologyBuilder.build(messageQueue.getTopic()); this.mq2Processor.put(key, processor); } } } private void removeTask(Set removeQueues) { for (MessageQueue removeQueue : removeQueues) { String key = Utils.buildKey(removeQueue.getBrokerName(), removeQueue.getTopic(), removeQueue.getQueueId()); mq2Processor.remove(key); } } @SuppressWarnings("unchecked") Processor selectProcessor(String key) { return (Processor) this.mq2Processor.get(key); } public void setRecoverHandler(BiFunction, Set, Throwable> handler) { this.recoverHandler = handler; } public void setResetOffsetHandler(Function, Throwable> handler) { this.resetOffsetHandler = handler; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/Processor.java ================================================ package org.apache.rocketmq.streams.core.running; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable; public interface Processor { void addChild(Processor processor); void preProcess(StreamContext context) throws RecoverStateStoreThrowable; void process(T data) throws Throwable; } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/RocketMQClient.java ================================================ package org.apache.rocketmq.streams.core.running; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer; import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.producer.DefaultMQProducer; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.protocol.route.QueueData; import org.apache.rocketmq.common.protocol.route.TopicRouteData; import org.apache.rocketmq.remoting.exception.RemotingException; import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Set; import java.util.UUID; import static org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData.SUB_ALL; import static org.apache.rocketmq.streams.core.common.Constant.*; public class RocketMQClient { private static final Logger logger = LoggerFactory.getLogger(RocketMQClient.class); private final String nameSrvAddr; public RocketMQClient(String nameSrvAddr) { this.nameSrvAddr = nameSrvAddr; } public DefaultLitePullConsumer pullConsumer(String groupName, Set topics) throws MQClientException { DefaultLitePullConsumer pullConsumer = new DefaultLitePullConsumer(groupName); pullConsumer.setNamesrvAddr(nameSrvAddr); pullConsumer.setAutoCommit(false); pullConsumer.setPullBatchSize(1000); for (String topic : topics) { pullConsumer.subscribe(topic, SUB_ALL); logger.debug("subscribe topic:{}, groupName:{}", topic, groupName); } return pullConsumer; } public DefaultMQProducer producer(String groupName) { DefaultMQProducer producer = new DefaultMQProducer(groupName); producer.setNamesrvAddr(nameSrvAddr); return producer; } public DefaultMQAdminExt getMQAdmin() throws MQClientException { DefaultMQAdminExt mqAdminExt = new DefaultMQAdminExt(1000); mqAdminExt.setInstanceName(UUID.randomUUID().toString()); mqAdminExt.setNamesrvAddr(nameSrvAddr); mqAdminExt.start(); return mqAdminExt; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/StreamContext.java ================================================ package org.apache.rocketmq.streams.core.running; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.client.producer.DefaultMQProducer; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.state.StateStore; import org.apache.rocketmq.streams.core.window.fire.IdleWindowScaner; import java.util.List; import java.util.Properties; public interface StreamContext { void init(List> childrenProcessors); StateStore getStateStore(); long getDataTime(); K getKey(); Properties getUserProperties(); Properties getHeader(); DefaultMQProducer getDefaultMQProducer(); String getSourceBrokerName(); String getSourceTopic(); Integer getSourceQueueId(); IdleWindowScaner getDefaultWindowScaner(); StreamContext copy(); void forward(Data data) throws Throwable; } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/StreamContextImpl.java ================================================ package org.apache.rocketmq.streams.core.running; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.client.producer.DefaultMQProducer; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.state.StateStore; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.streams.core.window.fire.IdleWindowScaner; import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; import java.util.Properties; /** * 1、可以获得当前processor; * 2、可以获得下一个执行节点 * 3、可获得动态的运行时信息,例如正在处理的数据来自那个topic,MQ,偏移量多少; */ public class StreamContextImpl implements StreamContext { private static final Logger logger = LoggerFactory.getLogger(StreamContextImpl.class); private final Properties properties; private final DefaultMQProducer producer; private final DefaultMQAdminExt mqAdmin; private final StateStore stateStore; private final String messageFromWhichSourceTopicQueue; private final IdleWindowScaner idleWindowScaner; private Object key; private long dataTime; private Properties header = new Properties(); private final List> childList = new ArrayList<>(); StreamContextImpl(Properties properties, DefaultMQProducer producer, DefaultMQAdminExt mqAdmin, StateStore stateStore, String messageFromWhichSourceTopicQueue, IdleWindowScaner idleWindowScaner) { this.properties = properties; this.producer = producer; this.mqAdmin = mqAdmin; this.stateStore = stateStore; this.messageFromWhichSourceTopicQueue = messageFromWhichSourceTopicQueue; this.idleWindowScaner = idleWindowScaner; } @Override public void init(List> childrenProcessors) { this.childList.clear(); if (childrenProcessors != null) { this.childList.addAll(childrenProcessors); } } @Override public StateStore getStateStore() { return this.stateStore; } @Override public DefaultMQProducer getDefaultMQProducer() { return producer; } public String getSourceBrokerName() { String[] split = Utils.split(messageFromWhichSourceTopicQueue); return split[0]; } public String getSourceTopic() { String[] split = Utils.split(messageFromWhichSourceTopicQueue); return split[1]; } public Integer getSourceQueueId() { String[] split = Utils.split(messageFromWhichSourceTopicQueue); return Integer.parseInt(split[2]); } @Override public long getDataTime() { return this.dataTime; } @Override @SuppressWarnings("unchecked") public K getKey() { return (K) key; } void setKey(K key) { this.key = key; } @Override public Properties getUserProperties() { Properties result = new Properties(); result.putAll(this.properties); return result; } @Override public Properties getHeader() { Properties result = new Properties(); result.putAll(this.header); return result; } @Override public IdleWindowScaner getDefaultWindowScaner() { return this.idleWindowScaner; } @Override public StreamContext copy() { StreamContextImpl streamContext = new StreamContextImpl<>(this.properties, this.producer, this.mqAdmin, this.stateStore, this.messageFromWhichSourceTopicQueue, this.idleWindowScaner); streamContext.key = this.key; streamContext.dataTime = this.dataTime; streamContext.header = new Properties(this.header); streamContext.childList.addAll(this.childList); return streamContext; } @Override public void forward(Data data) throws Throwable { this.key = data.getKey(); if (data.getTimestamp() != null) { this.dataTime = data.getTimestamp(); } this.header = data.getHeader(); List> store = new ArrayList<>(childList); for (Processor processor : childList) { try { processor.preProcess(this); processor.process(data.getValue()); } finally { this.childList.clear(); this.childList.addAll(store); } } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/WorkerThread.java ================================================ package org.apache.rocketmq.streams.core.running; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer; import org.apache.rocketmq.client.consumer.MessageQueueListener; import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.producer.DefaultMQProducer; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.admin.ConsumeStats; import org.apache.rocketmq.common.admin.OffsetWrapper; 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.body.ClusterInfo; import org.apache.rocketmq.common.protocol.route.BrokerData; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.DataProcessThrowable; import org.apache.rocketmq.streams.core.exception.RStreamsException; import org.apache.rocketmq.streams.core.function.supplier.SourceSupplier; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.metadata.StreamConfig; import org.apache.rocketmq.streams.core.window.fire.IdleWindowScaner; import org.apache.rocketmq.streams.core.window.TimeType; import org.apache.rocketmq.streams.core.state.RocketMQStore; import org.apache.rocketmq.streams.core.state.RocksDBStore; import org.apache.rocketmq.streams.core.state.StateStore; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.RocketMQUtil; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import static org.apache.rocketmq.streams.core.common.Constant.*; import static org.apache.rocketmq.streams.core.metadata.StreamConfig.ROCKETMQ_STREAMS_CONSUMER_FORM_WHERE; import static org.apache.rocketmq.streams.core.metadata.StreamConfig.ROCKETMQ_STREAMS_CONSUMER_GROUP; public class WorkerThread extends Thread { private static final Logger logger = LoggerFactory.getLogger(WorkerThread.class.getName()); private final TopologyBuilder topologyBuilder; private final PlanetaryEngine planetaryEngine; private final Properties properties; private final String jobId; private final ScheduledExecutorService executor; public WorkerThread(String threadName, TopologyBuilder topologyBuilder, Properties properties, ScheduledExecutorService executor) throws MQClientException { super(threadName); this.topologyBuilder = topologyBuilder; this.properties = properties; jobId = topologyBuilder.getJobId(); this.executor = executor; String groupName = String.join("_", jobId, ROCKETMQ_STREAMS_CONSUMER_GROUP); RocketMQClient rocketMQClient = new RocketMQClient(properties.getProperty(MixAll.NAMESRV_ADDR_PROPERTY)); Set topicNames = topologyBuilder.getSourceTopic(); DefaultLitePullConsumer unionConsumer = rocketMQClient.pullConsumer(groupName, topicNames); MessageQueueListener originListener = unionConsumer.getMessageQueueListener(); MessageQueueListenerWrapper wrapper = new MessageQueueListenerWrapper(originListener, topologyBuilder); unionConsumer.setMessageQueueListener(wrapper); DefaultMQProducer producer = rocketMQClient.producer(groupName); DefaultMQAdminExt mqAdmin = rocketMQClient.getMQAdmin(); RocksDBStore rocksDBStore = new RocksDBStore(threadName); RocketMQStore store = new RocketMQStore(producer, rocksDBStore, mqAdmin, this.properties); this.planetaryEngine = new PlanetaryEngine<>(unionConsumer, producer, store, mqAdmin, wrapper, topicNames); } @Override public void run() { try { this.planetaryEngine.start(); logger.info("worker thread=[{}], start task success, jobId:{}", this.getName(), jobId); this.planetaryEngine.maybeResetOffsetToFirst(); this.planetaryEngine.runInLoop(); } catch (Throwable e) { logger.error("worker thread=[{}], error:{}.", this.getName(), e); throw new RStreamsException(e); } finally { this.planetaryEngine.stop(); } } public void shutdown() { this.planetaryEngine.stop(); } @SuppressWarnings("unchecked") class PlanetaryEngine { private final DefaultLitePullConsumer unionConsumer; private final DefaultMQProducer producer; private final DefaultMQAdminExt mqAdmin; private final StateStore stateStore; private final MessageQueueListenerWrapper wrapper; private final IdleWindowScaner idleWindowScaner; private volatile boolean stop = false; private Set sourceTopicSet; private final HashSet mq2Commit = new HashSet<>(); public PlanetaryEngine(DefaultLitePullConsumer unionConsumer, DefaultMQProducer producer, StateStore stateStore, DefaultMQAdminExt mqAdmin, MessageQueueListenerWrapper wrapper, Set sourceTopicSet) { this.unionConsumer = unionConsumer; this.producer = producer; this.mqAdmin = mqAdmin; this.stateStore = stateStore; this.wrapper = wrapper; this.wrapper.setRecoverHandler((addQueue, removeQueue) -> { try { PlanetaryEngine.this.stateStore.recover(addQueue, removeQueue); return null; } catch (Throwable e) { logger.error("recover error.", e); return e; } }); this.sourceTopicSet = sourceTopicSet; Integer idleTime = (Integer) WorkerThread.this.properties.getOrDefault(StreamConfig.IDLE_TIME_TO_FIRE_WINDOW, 2000); int commitInterval = (Integer) WorkerThread.this.properties.getOrDefault(StreamConfig.COMMIT_STATE_INTERNAL_MS, 2 * 1000); this.idleWindowScaner = new IdleWindowScaner(idleTime, executor); WorkerThread.this.executor.scheduleAtFixedRate(() -> { try { doCommit(mq2Commit); } catch (Throwable t) { logger.error("commit offset and state error.", t); } }, 1000, commitInterval, TimeUnit.MILLISECONDS); } void start() throws Throwable { createShuffleTopic(); this.unionConsumer.start(); this.producer.start(); this.stateStore.init(); } void runInLoop() throws Throwable { while (!stop) { try { List list = this.unionConsumer.poll(10); for (MessageExt messageExt : list) { byte[] body = messageExt.getBody(); if (body == null || body.length == 0) { break; } String keyClassName = messageExt.getUserProperty(Constant.SHUFFLE_KEY_CLASS_NAME); String valueClassName = messageExt.getUserProperty(Constant.SHUFFLE_VALUE_CLASS_NAME); String topic = messageExt.getTopic(); int queueId = messageExt.getQueueId(); String brokerName = messageExt.getBrokerName(); MessageQueue queue = new MessageQueue(topic, brokerName, queueId); mq2Commit.add(queue); logger.debug("source topic queue:[{}]", queue); String key = Utils.buildKey(brokerName, topic, queueId); SourceSupplier.SourceProcessor processor = (SourceSupplier.SourceProcessor) wrapper.selectProcessor(key); StreamContextImpl context = new StreamContextImpl<>(properties, producer, mqAdmin, stateStore, key, idleWindowScaner); processor.preProcess(context); Pair pair = processor.deserialize(keyClassName, valueClassName, body); long timestamp = prepareTime(messageExt, processor); Data data = new Data<>(pair.getKey(), pair.getValue(), timestamp, new Properties()); context.setKey(pair.getKey()); if (topic.contains(Constant.SHUFFLE_TOPIC_SUFFIX)) { logger.debug("shuffle data: [{}]", data); } else { logger.debug("source data: [{}]", data); } try { context.forward(data); } catch (Throwable t) { logger.error("process error.", t); throw new DataProcessThrowable(t); } } } catch (Throwable t) { Object skipDataError = properties.getOrDefault(Constant.SKIP_DATA_ERROR, Boolean.TRUE); if (skipDataError == Boolean.TRUE) { logger.error("ignore error, jobId=[{}], skip this data.", topologyBuilder.getJobId(), t); //ignored } else { throw t; } } } } void doCommit(HashSet set) throws Throwable { if (set != null && set.size() != 0) { this.stateStore.persist(set); this.unionConsumer.commit(set, true); for (MessageQueue messageQueue : set) { logger.debug("committed messageQueue: [{}]", messageQueue); } set.clear(); } } void maybeResetOffsetToFirst() throws Exception { ConsumeFromWhere consumeFromWhere = (ConsumeFromWhere) properties.getOrDefault(ROCKETMQ_STREAMS_CONSUMER_FORM_WHERE, ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET); if (!consumeFromWhere.equals(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET)) { return; } for (String topic : sourceTopicSet) { // 内部 topic 不能重置位点 if (topic.endsWith(Constant.SHUFFLE_TOPIC_SUFFIX) || topic.endsWith(STATE_TOPIC_SUFFIX)) { continue; } ConsumeStats consumeStats = mqAdmin.examineConsumeStats(unionConsumer.getConsumerGroup(), topic); Map offsetTable = consumeStats.getOffsetTable(); Set messageQueues = offsetTable.keySet(); for (MessageQueue messageQueue : messageQueues) { try { // 如果有消费进度,说明已经开始消费,跳过重置其消费进度 if (offsetTable.containsKey(messageQueue) && offsetTable.get(messageQueue).getConsumerOffset() != DEFAULT_CONSUME_OFFSET) { break; } Long minOffset = mqAdmin.minOffset(messageQueue); String brokerName = messageQueue.getBrokerName(); ClusterInfo clusterInfo = mqAdmin.examineBrokerClusterInfo(); BrokerData brokerData = clusterInfo.getBrokerAddrTable().get(brokerName); if (brokerData == null) { String msg = String.format("get broker error, have no broker info (name:%s)", brokerName); logger.error(msg); throw new RStreamsException(msg); } for (String brokerAddress : brokerData.getBrokerAddrs().values()) { mqAdmin.resetOffsetByQueueId(brokerAddress, unionConsumer.getConsumerGroup(), messageQueue.getTopic(), messageQueue.getQueueId(), minOffset); } } catch (Exception e) { logger.error("reset messageQueue:{} consumer offset to first failed.", messageQueue, e); throw e; } } } } long prepareTime(MessageExt messageExt, SourceSupplier.SourceProcessor processor) { TimeType type = (TimeType) properties.get(StreamConfig.TIME_TYPE); long timestamp; String userProperty = messageExt.getUserProperty(Constant.SOURCE_TIMESTAMP); if (!StringUtils.isEmpty(userProperty)) { //data come from shuffle topic timestamp = Long.parseLong(userProperty); } else { //data come from user source topic timestamp = processor.getTimestamp(messageExt, type); } return timestamp; } void createShuffleTopic() throws Throwable { Set total = WorkerThread.this.topologyBuilder.getSourceTopic(); List shuffleTopic = new ArrayList<>(); for (String topic : total) { if (topic.endsWith(Constant.SHUFFLE_TOPIC_SUFFIX)) { shuffleTopic.add(topic); } } for (String topicName : shuffleTopic) { RocketMQUtil.createStaticTopic(mqAdmin, topicName, StreamConfig.SHUFFLE_TOPIC_QUEUE_NUM); } } public synchronized void stop() { if (this.stop) { return; } this.stop = true; try { this.unionConsumer.shutdown(); this.stateStore.close(); this.idleWindowScaner.close(); this.producer.shutdown(); this.mqAdmin.shutdown(); logger.info("shutdown engine success, thread:{}, jobId:{}", WorkerThread.this.getName(), jobId); } catch (Throwable e) { logger.error("error when stop engin.", e); throw new RStreamsException(e); } } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/serialization/KeyValueDeserializer.java ================================================ /* * 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.core.serialization; import org.apache.rocketmq.streams.core.util.Pair; public interface KeyValueDeserializer { default void configure(Object... args) throws Throwable { } Pair deserialize(byte[] source) throws Throwable; } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/serialization/KeyValueSerializer.java ================================================ /* * 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.core.serialization; public interface KeyValueSerializer { byte[] serialize(KEY key, T data) throws Throwable; } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/serialization/ShuffleProtocol.java ================================================ /* * 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.core.serialization; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; import org.apache.rocketmq.streams.core.util.Pair; /** * shuffle data how to encode KV *
 * +-----------+---------------+-----------+-------------+
 * | Int(4)    | Int(4)        | key bytes | value bytes |
 * | key length| value length  |           |             |
 * +-----------+---------------+-----------+-------------+
 * 
*/ public class ShuffleProtocol { private final ByteBuf buf = Unpooled.buffer(16); public byte[] merge(byte[] keyBytes, byte[] valueBytes) { if (keyBytes == null || keyBytes.length ==0) { return valueBytes; } if (valueBytes == null || valueBytes.length ==0) { return keyBytes; } buf.writeInt(keyBytes.length); buf.writeInt(valueBytes.length); buf.writeBytes(keyBytes); buf.writeBytes(valueBytes); byte[] bytes = new byte[buf.readableBytes()]; buf.readBytes(bytes); buf.clear(); return bytes; } public Pair split(byte[] total) { ByteBuf byteBuf = Unpooled.wrappedBuffer(total); int keyLength = byteBuf.readInt(); int valueLength = byteBuf.readInt(); ByteBuf keyByteBuf = byteBuf.readBytes(keyLength); ByteBuf valueByteBuf = byteBuf.readBytes(valueLength); byte[] keyBytes = new byte[keyByteBuf.readableBytes()]; keyByteBuf.readBytes(keyBytes); byte[] valueBytes = new byte[valueByteBuf.readableBytes()]; valueByteBuf.readBytes(valueBytes); byteBuf.release(); keyByteBuf.release(); valueByteBuf.release(); return new Pair<>(keyBytes, valueBytes); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/serialization/deImpl/KVJsonDeserializer.java ================================================ /* * 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.core.serialization.deImpl; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.serialization.KeyValueDeserializer; import org.apache.rocketmq.streams.core.serialization.ShuffleProtocol; public class KVJsonDeserializer extends ShuffleProtocol implements KeyValueDeserializer { private final ObjectMapper objectMapper = new ObjectMapper(); private Class keyType; private Class valueType; public KVJsonDeserializer() { objectMapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES) .disable(DeserializationFeature.FAIL_ON_NULL_FOR_PRIMITIVES) .enable(DeserializationFeature.ACCEPT_EMPTY_STRING_AS_NULL_OBJECT) .enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS) .enable(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN) .setNodeFactory(JsonNodeFactory.withExactBigDecimals(true)); } @Override @SuppressWarnings("unchecked") public void configure(Object... args) throws Throwable { String keyClassName = (String) args[0]; if (!StringUtils.isEmpty(keyClassName)) { keyType = (Class) Class.forName(keyClassName); } String valueClassName = (String) args[1]; if (!StringUtils.isEmpty(valueClassName)) { valueType = (Class) Class.forName(valueClassName); } } @Override public Pair deserialize(byte[] total) throws Throwable { Pair pair = split(total); K key = null; byte[] keyBytes = pair.getKey(); if (keyBytes != null && keyBytes.length != 0) { key = objectMapper.readValue(keyBytes, keyType); } V value; byte[] valueBytes = pair.getValue(); value = objectMapper.readValue(valueBytes, valueType); return new Pair<>(key, value); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/serialization/serImpl/KVJsonSerializer.java ================================================ /* * 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.core.serialization.serImpl; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import org.apache.rocketmq.streams.core.serialization.KeyValueSerializer; import org.apache.rocketmq.streams.core.serialization.ShuffleProtocol; public class KVJsonSerializer extends ShuffleProtocol implements KeyValueSerializer { private final ObjectMapper objectMapper = new ObjectMapper(); public KVJsonSerializer() { objectMapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES) .disable(DeserializationFeature.FAIL_ON_NULL_FOR_PRIMITIVES) .enable(DeserializationFeature.ACCEPT_EMPTY_STRING_AS_NULL_OBJECT) .enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS) .enable(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN) .setNodeFactory(JsonNodeFactory.withExactBigDecimals(true)); } @Override public byte[] serialize(K key, V value) throws Throwable { byte[] keyBytes; if (key == null) { keyBytes = new byte[0]; } else if (key instanceof byte[]) { keyBytes = (byte[]) key; } else { keyBytes = objectMapper.writeValueAsBytes(key); } byte[] valueBytes; if (value == null) { valueBytes = new byte[0]; } else if (value instanceof byte[]) { valueBytes = (byte[]) value; } else { valueBytes = objectMapper.writeValueAsBytes(value); } if (keyBytes.length == 0 && valueBytes.length == 0) { return new byte[0]; } return merge(keyBytes, valueBytes); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/state/AbstractStore.java ================================================ package org.apache.rocketmq.streams.core.state; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.util.Utils; import java.util.Arrays; import java.util.HashSet; import java.util.Iterator; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; public abstract class AbstractStore { private final Wrapper wrapper = new Wrapper(); protected void putInRecover(String stateTopicQueueKey, byte[] key) { wrapper.putInRecover(stateTopicQueueKey, key); } protected void putInCalculating(String stateTopicQueueKey, byte[] key) { wrapper.putInCalculating(stateTopicQueueKey, key); } protected Set getInCalculating(String stateTopicQueue) { return wrapper.getInCalculating(stateTopicQueue); } protected void removeCalculating(String stateTopicQueue) { wrapper.removeCalculating(stateTopicQueue); } protected Set getAll(String stateTopicQueue) { return wrapper.getAll(stateTopicQueue); } protected String whichStateTopicQueueBelongTo(byte[] key) { return wrapper.whichStateTopicQueueBelongTo(key); } protected void removeAllKey(byte[] key) { wrapper.deleteByKey(key); } protected void removeAll(String stateTopicQueue) { wrapper.removeAll(stateTopicQueue); } protected MessageQueue convertSourceTopicQueue2StateTopicQueue(MessageQueue messageQueue) { HashSet messageQueues = new HashSet<>(); messageQueues.add(messageQueue); Set stateTopicQueue = convertSourceTopicQueue2StateTopicQueue(messageQueues); Iterator iterator = stateTopicQueue.iterator(); return iterator.next(); } protected Set convertSourceTopicQueue2StateTopicQueue(Set messageQueues) { if (messageQueues == null || messageQueues.size() == 0) { return new HashSet<>(); } HashSet result = new HashSet<>(); for (MessageQueue messageQueue : messageQueues) { if (messageQueue.getTopic().endsWith(Constant.STATE_TOPIC_SUFFIX)) { result.add(messageQueue); continue; } MessageQueue queue = new MessageQueue(messageQueue.getTopic() + Constant.STATE_TOPIC_SUFFIX, messageQueue.getBrokerName(), messageQueue.getQueueId()); result.add(queue); } return result; } protected static String stateTopic2SourceTopic(String stateTopic) { if (StringUtils.isEmpty(stateTopic)) { return null; } return stateTopic.substring(0, stateTopic.lastIndexOf(Constant.STATE_TOPIC_SUFFIX)); } protected String buildKey(MessageExt messageExt) { return Utils.buildKey(messageExt.getBrokerName(), messageExt.getTopic(), messageExt.getQueueId()); } protected String buildKey(MessageQueue messageQueue) { return Utils.buildKey(messageQueue.getBrokerName(), messageQueue.getTopic(), messageQueue.getQueueId()); } static class Wrapper { //新增,写消费未提交保存的中间状态,提交时移除 private final ConcurrentHashMap> calculating = new ConcurrentHashMap<>(); //全量, 与rocksdb保持同步 private final ConcurrentHashMap> recover = new ConcurrentHashMap<>(); public void putInRecover(String stateTopicQueueKey, byte[] key) { Set allSet = this.recover.computeIfAbsent(stateTopicQueueKey, s -> new HashSet<>()); for (byte[] item : allSet) { if (Arrays.equals(item, key)) { return; } } allSet.add(key); } public void putInCalculating(String stateTopicQueueKey, byte[] key) { Set keySet = this.calculating.computeIfAbsent(stateTopicQueueKey, s -> new HashSet<>()); for (byte[] item : keySet) { if (Arrays.equals(item, key)) { return; } } keySet.add(key); putInRecover(stateTopicQueueKey, key); } public Set getInCalculating(String stateTopicQueue) { return calculating.getOrDefault(stateTopicQueue, new HashSet<>()); } public Set getAll(String stateTopicQueue) { Set calculating = this.calculating.getOrDefault(stateTopicQueue, new HashSet<>()); Set recover = this.recover.getOrDefault(stateTopicQueue, new HashSet<>()); Set result = new HashSet<>(); result.addAll(calculating); result.addAll(recover); //可能有重复,不同byte[] 但是时一个key return result; } public String whichStateTopicQueueBelongTo(byte[] key) { for (String uniqueQueue : recover.keySet()) { for (byte[] tempKeyByte : recover.getOrDefault(uniqueQueue, new HashSet<>())) { if (Arrays.equals(tempKeyByte, key)) { return uniqueQueue; } } } for (String uniqueQueue : calculating.keySet()) { for (byte[] tempKeyByte : calculating.getOrDefault(uniqueQueue, new HashSet<>())) { if (Arrays.equals(tempKeyByte, key)) { return uniqueQueue; } } } return null; } public void deleteByKey(byte[] key) { { Set>> entries = calculating.entrySet(); Iterator>> iterator = entries.iterator(); while (iterator.hasNext()) { Map.Entry> next = iterator.next(); Set keySet = next.getValue(); if (keySet != null) { keySet.removeIf(rocksDBKey -> Arrays.equals(rocksDBKey, key)); if (keySet.size() == 0) { iterator.remove(); } } } } { Set>> entries = recover.entrySet(); Iterator>> iterator = entries.iterator(); while (iterator.hasNext()) { Map.Entry> next = iterator.next(); Set keySet = next.getValue(); if (keySet != null) { keySet.removeIf(rocksDBKey -> Arrays.equals(rocksDBKey, key)); if (keySet.size() == 0) { iterator.remove(); } } } } } public void removeCalculating(String stateTopicQueueKey) { this.calculating.remove(stateTopicQueueKey); } public void removeAll(String stateTopicQueueKey) { this.recover.remove(stateTopicQueueKey); this.calculating.remove(stateTopicQueueKey); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/state/RocketMQStore.java ================================================ package org.apache.rocketmq.streams.core.state; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer; import org.apache.rocketmq.client.producer.DefaultMQProducer; import org.apache.rocketmq.common.CountDownLatch2; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.protocol.route.BrokerData; import org.apache.rocketmq.common.protocol.route.QueueData; import org.apache.rocketmq.common.protocol.route.TopicRouteData; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.metadata.StreamConfig; import org.apache.rocketmq.streams.core.util.ColumnFamilyUtil; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.serialization.ShuffleProtocol; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.RocketMQUtil; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; public class RocketMQStore extends AbstractStore implements StateStore { private static final Logger logger = LoggerFactory.getLogger(RocketMQStore.class.getName()); private final DefaultMQProducer producer; private final DefaultMQAdminExt mqAdmin; private final RocksDBStore rocksDBStore; private final Properties properties; private final ExecutorService executor = Executors.newFixedThreadPool(8); private final ShuffleProtocol protocol = new ShuffleProtocol(); private final ConcurrentHashMap recoveringQueueMutex = new ConcurrentHashMap<>(); public RocketMQStore(DefaultMQProducer producer, RocksDBStore rocksDBStore, DefaultMQAdminExt mqAdmin, Properties properties) { this.producer = producer; this.mqAdmin = mqAdmin; this.rocksDBStore = rocksDBStore; this.properties = properties; } @Override public void init() throws Throwable { } @Override public void recover(Set addQueues, Set removeQueues) throws Throwable { this.loadState(addQueues); this.removeState(removeQueues); } @Override public void waitIfNotReady(MessageQueue messageQueue) throws RecoverStateStoreThrowable { MessageQueue stateTopicQueue = convertSourceTopicQueue2StateTopicQueue(messageQueue); CountDownLatch2 waitPoint = this.recoveringQueueMutex.get(stateTopicQueue); long start = 0; long end = 0; try { start = System.currentTimeMillis(); waitPoint.await(5000, TimeUnit.MILLISECONDS); end = System.currentTimeMillis(); } catch (Throwable t) { throw new RecoverStateStoreThrowable(t); } finally { long cost = end - start; if (cost > 2000) { logger.error("recover finish, consume time:" + cost + " ms."); } } } @Override public byte[] get(String columnFamily, byte[] key) throws Throwable { if (key == null || key.length == 0) { return new byte[0]; } return this.rocksDBStore.get(columnFamily, key); } @Override public void put(MessageQueue stateTopicMessageQueue, String columnFamily, byte[] key, byte[] value) throws Throwable { String stateTopicQueueKey = buildKey(stateTopicMessageQueue); super.putInCalculating(stateTopicQueueKey, key); this.rocksDBStore.put(columnFamily, key, value); } @Override public List> searchStateLessThanWatermark(String keyPrefix, long lessThanThisTime, ValueMapperAction deserializer) throws Throwable { if (StringUtils.isEmpty(keyPrefix)) { return new ArrayList<>(); } return this.rocksDBStore.searchStateLessThanWatermark(keyPrefix, lessThanThisTime, deserializer); } @Override public List> searchByKeyPrefix(String keyPrefix, ValueMapperAction string2Bytes, ValueMapperAction byte2String) throws Throwable { if (StringUtils.isEmpty(keyPrefix)) { return new ArrayList<>(); } return this.rocksDBStore.searchByKeyPrefix(keyPrefix, string2Bytes, byte2String); } @Override public void delete(byte[] key) throws Throwable { if (key == null || key.length == 0) { return; } //删除远程 String stateTopicQueue = super.whichStateTopicQueueBelongTo(key); String[] split = Utils.split(stateTopicQueue); String topic = split[1]; MessageQueue queue = new MessageQueue(split[1], split[0], Integer.parseInt(split[2])); Message message = new Message(topic, Constant.EMPTY_BODY.getBytes(StandardCharsets.UTF_8)); message.setKeys(Utils.toHexString(key)); message.putUserProperty(Constant.SHUFFLE_KEY_CLASS_NAME, key.getClass().getName()); message.putUserProperty(Constant.EMPTY_BODY, Constant.TRUE); producer.send(message, queue); //删除rocksdb this.rocksDBStore.deleteByKey(ColumnFamilyUtil.getColumnFamilyByKey(key), key); //删除内存中的key super.removeAllKey(key); logger.debug("delete key from RocketMQ and Rocksdb, key=" + new String(key, StandardCharsets.UTF_8) + ",MessageQueue: " + queue); } @Override public void persist(Set messageQueues) throws Throwable { if (messageQueues == null || messageQueues.size() == 0) { return; } Set stateTopicQueues = convertSourceTopicQueue2StateTopicQueue(messageQueues); for (MessageQueue stateTopicQueue : stateTopicQueues) { String stateTopicQueueKey = buildKey(stateTopicQueue); Set keySet = super.getInCalculating(stateTopicQueueKey); if (keySet == null || keySet.size() == 0) { continue; } String stateTopic = stateTopicQueue.getTopic(); boolean isStaticTopic = stateTopicQueue.getBrokerName().equals(Constant.STATIC_TOPIC_BROKER_NAME); createStateTopic(stateTopic, isStaticTopic); for (byte[] key : keySet) { byte[] valueBytes = this.rocksDBStore.get(ColumnFamilyUtil.getColumnFamilyByKey(key), key); if (valueBytes == null) { continue; } byte[] body = this.protocol.merge(key, valueBytes); Message message = new Message(stateTopicQueue.getTopic(), body); message.setKeys(Utils.toHexString(key)); try { logger.debug("persist key: " + new String(key, StandardCharsets.UTF_8) + ",messageQueue: " + stateTopicQueue); } catch (Throwable t) { //key is not string, maybe. } this.producer.send(message, stateTopicQueue); } super.removeCalculating(stateTopicQueueKey); } } public void loadState(Set addQueues) throws Throwable { if (addQueues == null || addQueues.size() == 0) { return; } final DefaultLitePullConsumer consumer = new DefaultLitePullConsumer(StreamConfig.ROCKETMQ_STREAMS_STATE_CONSUMER_GROUP); consumer.setNamesrvAddr(properties.getProperty(MixAll.NAMESRV_ADDR_PROPERTY)); consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); consumer.setAutoCommit(false); consumer.start(); Set stateTopicQueue = convertSourceTopicQueue2StateTopicQueue(addQueues); for (MessageQueue messageQueue : stateTopicQueue) { createStateTopic(messageQueue.getTopic(), messageQueue.getBrokerName().equals(Constant.STATIC_TOPIC_BROKER_NAME)); } consumer.assign(stateTopicQueue); for (MessageQueue queue : stateTopicQueue) { consumer.seekToBegin(queue); } Future future = this.executor.submit(() -> { try { pullToLast(consumer); } catch (Throwable e) { logger.error("pull to last error.", e); throw new RuntimeException(e); } finally { consumer.shutdown(); } }); try { future.get(100, TimeUnit.MILLISECONDS); } catch (InterruptedException | TimeoutException e) { } } public void removeState(Set removeQueues) throws Throwable { if (removeQueues == null || removeQueues.size() == 0) { return; } Future future = this.executor.submit(() -> { try { if (removeQueues.size() == 0) { return; } Set stateTopicQueue = convertSourceTopicQueue2StateTopicQueue(removeQueues); Map> groupByUniqueQueue = stateTopicQueue.stream().parallel().collect(Collectors.groupingBy(this::buildKey)); for (String stateUniqueQueue : groupByUniqueQueue.keySet()) { Set stateTopicQueueKey = super.getAll(stateUniqueQueue); for (byte[] key : stateTopicQueueKey) { this.rocksDBStore.deleteByKey(ColumnFamilyUtil.getColumnFamilyByKey(key), key); } super.removeAll(stateUniqueQueue); } for (MessageQueue stateMessageQueue : stateTopicQueue) { this.recoveringQueueMutex.remove(stateMessageQueue); } } catch (Throwable e) { logger.error("remove state error", e); throw new RuntimeException(e); } }); try { future.get(100, TimeUnit.MILLISECONDS); } catch (InterruptedException | TimeoutException e) { } } private void pullToLast(DefaultLitePullConsumer consumer) throws Throwable { Set readyToRecover = consumer.assignment(); for (MessageQueue messageQueue : readyToRecover) { this.recoveringQueueMutex.computeIfAbsent(messageQueue, messageQueue1 -> new CountDownLatch2(1)); } List holder = new ArrayList<>(); //recover List result = consumer.poll(50); while (result != null && result.size() != 0) { holder.addAll(result); if (holder.size() <= 1000) { result = consumer.poll(50); continue; } replayState(holder); holder.clear(); result = consumer.poll(50); } if (holder.size() != 0) { replayState(holder); } //恢复完毕; Set recoverOver = consumer.assignment(); for (MessageQueue messageQueue : recoverOver) { CountDownLatch2 waitPoint = this.recoveringQueueMutex.get(messageQueue); waitPoint.countDown(); } } //拉的数据越多,重放效率越高, // 能保证一个q里面后面pull到的数据queueOffset一定比前一批次拉取的queueOffset大吗? private void replayState(List msgs) throws Throwable { if (msgs == null || msgs.size() == 0) { return; } Map> groupByQueueId = msgs.stream().parallel().collect(Collectors.groupingBy(this::buildKey)); for (String uniqueQueue : groupByQueueId.keySet()) { List messageExts = groupByQueueId.get(uniqueQueue); Map> groupByKeyHashcode = messageExts.stream().parallel().collect(Collectors.groupingBy(MessageExt::getKeys)); for (String keyHashcode : groupByKeyHashcode.keySet()) { //相同brokerName@topic@queueId + keyHashcode 在一次拉取中的所有数据 List exts = groupByKeyHashcode.get(keyHashcode); //取最大queueOffset的消息,按照queueOffset,相同key,大的queueOffset覆盖小的queueOffset MessageExt result = exts.stream() .max(Comparator.comparingLong(MessageExt::getQueueOffset)) .orElse(null); if (result == null) { continue; } String emptyBody = result.getUserProperty(Constant.EMPTY_BODY); if (Constant.TRUE.equals(emptyBody)) { continue; } byte[] body = result.getBody(); Pair pair = this.protocol.split(body); byte[] key = pair.getKey(); byte[] value = pair.getValue(); //放入rocksdb MessageQueue stateTopicQueue = new MessageQueue(result.getTopic(), result.getBrokerName(), result.getQueueId()); try { logger.debug("recover state, key: " + new String(key, StandardCharsets.UTF_8) + ", stateTopicQueue: " + stateTopicQueue); } catch (Throwable t) { } String stateTopicQueueKey = buildKey(stateTopicQueue); super.putInRecover(stateTopicQueueKey, key); this.rocksDBStore.put(ColumnFamilyUtil.getColumnFamilyByKey(key), key, value); } } } private void createStateTopic(String stateTopic, boolean sourceTopicIsStaticTopic) throws Exception { if (RocketMQUtil.checkWhetherExist(stateTopic)) { return; } String sourceTopic = stateTopic2SourceTopic(stateTopic); Pair> clustersPair = getTotalQueueNumAndClusters(sourceTopic); if (sourceTopicIsStaticTopic) { RocketMQUtil.createStaticCompactTopic(mqAdmin, stateTopic, clustersPair.getKey(), clustersPair.getValue()); } else { RocketMQUtil.createNormalTopic(mqAdmin, sourceTopic, stateTopic); } } private Pair> getTotalQueueNumAndClusters(String sourceTopic) throws Exception { int queueNum = 0; //找到brokerAddr TopicRouteData topicRouteData = mqAdmin.examineTopicRouteInfo(sourceTopic); List queueData = topicRouteData.getQueueDatas(); List brokerData = topicRouteData.getBrokerDatas(); Set clusterSet = brokerData.stream().collect(Collectors.groupingBy(BrokerData::getCluster)).keySet(); for (QueueData data : queueData) { //只看readQueue queueNum += data.getReadQueueNums(); } return new Pair>(queueNum, clusterSet); } @Override public void close() throws Exception { this.rocksDBStore.close(); this.executor.shutdown(); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/state/RocksDBStore.java ================================================ package org.apache.rocketmq.streams.core.state; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.util.ColumnFamilyUtil; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.Utils; import org.rocksdb.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; public class RocksDBStore extends AbstractStore implements AutoCloseable { private static final Logger logger = LoggerFactory.getLogger(RocksDBStore.class); private static final String ROCKSDB_PATH = "/tmp/rocksdb"; private RocksDB rocksDB; private WriteOptions writeOptions; private ReadOptions readOptions; private File storeFile; public RocksDBStore(String path) { createRocksDB(path); } private void createRocksDB(String path) { try (final Options options = new Options().setCreateIfMissing(true)) { try { String rocksdbFilePath = String.format("%s/%s", ROCKSDB_PATH, path); storeFile = new File(rocksdbFilePath); if (storeFile.exists()) { FileUtils.forceDelete(storeFile); } if (!storeFile.mkdirs()) { throw new RuntimeException("before create rocksdb,mkdir path " + rocksdbFilePath + " error"); } this.rocksDB = TtlDB.open(options, rocksdbFilePath, 10800, false); ColumnFamilyUtil.createColumnFamilies(this.rocksDB, new ColumnFamilyOptions()); writeOptions = new WriteOptions(); writeOptions.setSync(false); writeOptions.setDisableWAL(true); } catch (RocksDBException e) { throw new RuntimeException("create rocksdb error " + e.getMessage()); } catch (IOException e) { throw new RuntimeException("delete rocksdb directory:" + ROCKSDB_PATH + "field."); } } } public byte[] get(String columnFamilyName, byte[] key) throws RocksDBException { if (key == null) { return null; } return rocksDB.get(ColumnFamilyUtil.getColumnFamilyHandleByName(columnFamilyName), key); } public void put(String columnFamilyName, byte[] key, byte[] value) throws RocksDBException { rocksDB.put(ColumnFamilyUtil.getColumnFamilyHandleByName(columnFamilyName), writeOptions, key, value); } public List> searchStateLessThanWatermark(String name, long lessThanThisTime, ValueMapperAction deserializer) throws Throwable { readOptions = new ReadOptions(); readOptions.setPrefixSameAsStart(true).setTotalOrderSeek(true); RocksIterator rocksIterator = rocksDB.newIterator(ColumnFamilyUtil.getColumnFamilyHandleByName(ColumnFamilyUtil.WINDOW_STATE_CF), readOptions); byte[] keyBytePrefix = name.getBytes(StandardCharsets.UTF_8); rocksIterator.seek(keyBytePrefix); List> temp = new ArrayList<>(); while (rocksIterator.isValid()) { byte[] keyBytes = rocksIterator.key(); byte[] valueBytes = rocksIterator.value(); rocksIterator.next(); WindowKey windowKey = deserializer.convert(keyBytes); if (!windowKey.getOperatorName().equals(name)) { continue; } if (windowKey.getWindowEnd() >= lessThanThisTime) { continue; } Pair pair = new Pair<>(keyBytes, valueBytes); temp.add(pair); } return temp; } public List> searchByKeyPrefix(String keyPrefix, ValueMapperAction string2Bytes, ValueMapperAction byte2String) throws Throwable { readOptions = new ReadOptions(); readOptions.setPrefixSameAsStart(true).setTotalOrderSeek(true); RocksIterator rocksIterator = rocksDB.newIterator(readOptions); byte[] convert = string2Bytes.convert(keyPrefix); rocksIterator.seek(convert); List> temp = new ArrayList<>(); while (rocksIterator.isValid()) { byte[] keyBytes = rocksIterator.key(); byte[] valueBytes = rocksIterator.value(); if (skipWatermarkKey(keyBytes)) { continue; } String storeKey = byte2String.convert(keyBytes); if (storeKey.startsWith(keyPrefix)) { Pair pair = new Pair<>(storeKey, valueBytes); temp.add(pair); } rocksIterator.next(); } return temp; } public void deleteByKey(String columnFamilyName, byte[] key) throws RocksDBException { rocksDB.delete(ColumnFamilyUtil.getColumnFamilyHandleByName(columnFamilyName), key); } public void close() throws Exception { this.rocksDB.close(); if (this.storeFile != null && storeFile.exists()) { FileUtils.forceDelete(storeFile); logger.info("close RocksDB success, delete path:{}", storeFile.getPath()); } } //todo: column family to solve this problem. private boolean skipWatermarkKey(byte[] target) { if (target == null || target.length == 0) { return false; } try { String key = new String(target, StandardCharsets.UTF_8); return !StringUtils.isBlank(key) && key.startsWith(Constant.WATERMARK_KEY); } catch (Throwable ignored) { return false; } } public static void main(String[] args) throws Throwable { RocksDBStore rocksDBStore = new RocksDBStore("test"); String key = "time@1668249210000@1668249195000"; String key2 = "time@1668249210001@1668249195001"; Object value = "3"; Object value2 = "2"; byte[] keyBytes = Utils.object2Byte(key); byte[] valueBytes = Utils.object2Byte(value); byte[] keyBytes2 = Utils.object2Byte(key2); byte[] valueBytes2 = Utils.object2Byte(value2); rocksDBStore.put(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes2), keyBytes2, valueBytes2); rocksDBStore.put(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes), keyBytes, valueBytes); byte[] bytes = rocksDBStore.get(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes), keyBytes); Object result = Utils.byte2Object(bytes, Object.class); System.out.println(result); byte[] bytes2 = rocksDBStore.get(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes2), keyBytes2); Object result2 = Utils.byte2Object(bytes2, Object.class); System.out.println(result2); String keyPrefix = "time@1668249210000"; List> pairs = rocksDBStore.searchByKeyPrefix(keyPrefix, Utils::object2Byte, data -> Utils.byte2Object(data, String.class)); for (Pair pair : pairs) { assert pair.getKey().startsWith(keyPrefix); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/state/StateStore.java ================================================ package org.apache.rocketmq.streams.core.state; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.util.Pair; import java.util.List; import java.util.Set; public interface StateStore extends AutoCloseable { void init() throws Throwable; //addQueues messageQueue of source topic,removeQueues messageQueue of source topic void recover(Set addQueues, Set removeQueues) throws Throwable; //messageQueue check the state of source topic is ok, wait if not. void waitIfNotReady(MessageQueue messageQueue) throws RecoverStateStoreThrowable; byte[] get(String columnFamily, byte[] key) throws Throwable; void put(MessageQueue stateTopicMessageQueue, String columnFamily, byte[] key, byte[] value) throws Throwable; List> searchStateLessThanWatermark(String operatorName, long lessThanThisTime, ValueMapperAction deserializer) throws Throwable; List> searchByKeyPrefix(String keyPrefix, ValueMapperAction string2Bytes, ValueMapperAction byte2String) throws Throwable; void delete(byte[] key) throws Throwable; void persist(Set messageQueue) throws Throwable; } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/TopologyBuilder.java ================================================ package org.apache.rocketmq.streams.core.topology; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.topology.real.ProcessorFactory; import org.apache.rocketmq.streams.core.topology.real.RealProcessorFactory; import org.apache.rocketmq.streams.core.topology.real.SinkFactory; import org.apache.rocketmq.streams.core.topology.real.SourceFactory; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Set; import java.util.function.Supplier; public class TopologyBuilder { private final String jobId; private final LinkedHashMap> topic2SourceNodeFactory = new LinkedHashMap<>(); private final LinkedHashMap> realNodeFactory = new LinkedHashMap<>(); private final LinkedHashMap> topic2SinkNodeFactory = new LinkedHashMap<>(); private final HashMap> source2Group = new HashMap<>(); public TopologyBuilder(String jobId) { this.jobId = jobId; } public void addRealSource(String name, String topicName, Supplier> supplier) { SourceFactory sourceFactory = new SourceFactory<>(name, topicName, supplier); realNodeFactory.put(name, sourceFactory); topic2SourceNodeFactory.put(topicName, sourceFactory); //将source与sink之间的节点分为一个组,处理数据时,不同分组使用不同task source2Group.put(name, new ArrayList<>()); } public void addRealNode(String name, String parentName, Supplier> supplier) { RealProcessorFactory processorFactory = new ProcessorFactory<>(name, supplier); realNodeFactory.put(name, processorFactory); grouping(name, parentName); } public void addRealSink(String name, String parentName, String topicName, Supplier> supplier) { SinkFactory sinkFactory = new SinkFactory<>(name, supplier); realNodeFactory.put(name, sinkFactory); topic2SinkNodeFactory.put(topicName, sinkFactory); grouping(name, parentName); } private void grouping(String name, String parentName) { if (source2Group.containsKey(parentName)) { source2Group.get(parentName).add(name); } else { for (String sourceName : source2Group.keySet()) { List subsequentProcessor = source2Group.get(sourceName); if (subsequentProcessor.contains(parentName)) { subsequentProcessor.add(name); } } } } public Set getSourceTopic() { return Collections.unmodifiableSet(this.topic2SourceNodeFactory.keySet()); } @SuppressWarnings("unchecked") public Processor build(String topicName) { SourceFactory sourceFactory = (SourceFactory) topic2SourceNodeFactory.get(topicName); Processor sourceProcessor = sourceFactory.build(); String sourceName = sourceFactory.getName(); //集合中的顺序就是算子的父子顺序,前面的是后面的父亲节点 List groupNames = source2Group.get(sourceName); Processor parent = sourceProcessor; for (String child : groupNames) { RealProcessorFactory childProcessorFactory = (RealProcessorFactory) realNodeFactory.get(child); Processor childProcessor = childProcessorFactory.build(); parent.addChild(childProcessor); parent = childProcessor; } return sourceProcessor; } public String getJobId() { return jobId; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/real/ProcessorFactory.java ================================================ package org.apache.rocketmq.streams.core.topology.real; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.running.Processor; import java.util.ArrayList; import java.util.List; import java.util.function.Supplier; public class ProcessorFactory implements RealProcessorFactory { private final String name; private final Supplier> supplier; public ProcessorFactory(String name, Supplier> supplier) { this.name = name; this.supplier = supplier; } @Override public String getName() { return name; } @Override public Processor build() { return supplier.get(); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/real/RealProcessorFactory.java ================================================ package org.apache.rocketmq.streams.core.topology.real; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.running.Processor; import java.util.List; public interface RealProcessorFactory { String getName(); Processor build(); } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/real/SinkFactory.java ================================================ package org.apache.rocketmq.streams.core.topology.real; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.running.Processor; import java.util.ArrayList; import java.util.List; import java.util.function.Supplier; public class SinkFactory implements RealProcessorFactory { private final String name; private final List> children = new ArrayList<>(); private final Supplier> supplier; public SinkFactory(String name, Supplier> supplier) { this.name = name; this.supplier = supplier; } @Override public String getName() { return name; } @Override public Processor build() { return supplier.get(); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/real/SourceFactory.java ================================================ package org.apache.rocketmq.streams.core.topology.real; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.running.Processor; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.function.Supplier; public class SourceFactory implements RealProcessorFactory { private final String name; private final String topic; private final Supplier> supplier; public SourceFactory(String name, String topic, Supplier> supplier) { this.name = name; this.topic = topic; this.supplier = supplier; } @Override public String getName() { return name; } @Override public Processor build() { return supplier.get(); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/AbstractGraphNode.java ================================================ package org.apache.rocketmq.streams.core.topology.virtual; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Objects; public abstract class AbstractGraphNode implements GraphNode { private static final Logger logger = LoggerFactory.getLogger(AbstractGraphNode.class.getName()); private final List parents = new ArrayList<>(); private final List children = new ArrayList<>(); protected String name; public AbstractGraphNode(String name) { Objects.requireNonNull(name, "name can not be null."); this.name = name; } @Override public String getName() { return name; } @Override public void addParent(GraphNode graphNode) { if (!parents.contains(graphNode)) { parents.add(graphNode); } else { logger.error("GraphNode: [" + graphNode + "] has exist in parent set."); } } @Override public void addChild(GraphNode graphNode) { if (!children.contains(graphNode)) { children.add(graphNode); } else { logger.error("GraphNode: [" + graphNode + "] has exist in children set."); } } @Override public List getAllChild() { return Collections.unmodifiableList(this.children); } @Override public List getAllParent() { return Collections.unmodifiableList(this.parents); } @Override public boolean shuffleNode() { return false; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/GraphNode.java ================================================ package org.apache.rocketmq.streams.core.topology.virtual; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import java.util.List; public interface GraphNode { String getName(); void addParent(GraphNode graphNode); void addChild(GraphNode graphNode); List getAllChild(); List getAllParent(); boolean shuffleNode(); void addRealNode(TopologyBuilder builder); } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/ProcessorNode.java ================================================ package org.apache.rocketmq.streams.core.topology.virtual; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import java.util.ArrayList; import java.util.List; import java.util.function.Supplier; public class ProcessorNode extends AbstractGraphNode { protected final Supplier> supplier; protected final List parentNames; protected boolean shuffle = false; public ProcessorNode(String name, String parentName, Supplier> supplier) { super(name); this.supplier = supplier; this.parentNames = new ArrayList<>(); this.parentNames.add(parentName); } public ProcessorNode(String name, List parentNames, Supplier> supplier) { super(name); this.supplier = supplier; this.parentNames = parentNames; } public ProcessorNode(String name, List parentNames, boolean shuffle, Supplier> supplier) { super(name); this.supplier = supplier; this.parentNames = parentNames; this.shuffle = shuffle; } public ProcessorNode(String name, String parentName, boolean shuffle, Supplier> supplier) { super(name); this.supplier = supplier; this.parentNames = new ArrayList<>(); this.parentNames.add(parentName); this.shuffle = shuffle; } @Override public boolean shuffleNode() { return this.shuffle; } @Override public void addRealNode(TopologyBuilder builder) { //parentName数量大于2时,可能因为有的parent所在source并没有准备好,source2Group没有对应parent, // 无法添加导致该流最后缺少window以后的算子 for (String parentName : parentNames) { builder.addRealNode(name, parentName, supplier); } } @Override public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; ProcessorNode that = (ProcessorNode) o; return this.name.equals(that.name); } @Override public String toString() { return "ProcessorNode{" + "name=[" + name + "]}"; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/ShuffleProcessorNode.java ================================================ package org.apache.rocketmq.streams.core.topology.virtual; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.util.OperatorNameMaker; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.function.supplier.SinkSupplier; import org.apache.rocketmq.streams.core.function.supplier.SourceSupplier; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.serialization.deImpl.KVJsonDeserializer; import org.apache.rocketmq.streams.core.serialization.serImpl.KVJsonSerializer; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import java.util.function.Supplier; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SHUFFLE_SINK_PREFIX; import static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SHUFFLE_SOURCE_PREFIX; public class ShuffleProcessorNode extends ProcessorNode { public ShuffleProcessorNode(String name, String parentName, Supplier> supplier) { super(name, parentName, supplier); } @Override public void addRealNode(TopologyBuilder builder) { String topicName = name + Constant.SHUFFLE_TOPIC_SUFFIX; //join的时候需要一个相同的topicName String shuffleSinkName = OperatorNameMaker.makeName(SHUFFLE_SINK_PREFIX, builder.getJobId()); for (String parentName : parentNames) { builder.addRealSink(shuffleSinkName, parentName, topicName, new SinkSupplier<>(topicName, new KVJsonSerializer<>())); } String shuffleSourceName = OperatorNameMaker.makeName(SHUFFLE_SOURCE_PREFIX, builder.getJobId()); builder.addRealSource(shuffleSourceName, topicName, new SourceSupplier<>(topicName, new KVJsonDeserializer<>())); builder.addRealNode(name, shuffleSourceName, supplier); } @Override public String toString() { return "ShuffleProcessorNode{" + "name=[" + name + "]}"; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/SinkGraphNode.java ================================================ package org.apache.rocketmq.streams.core.topology.virtual; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import java.util.function.Supplier; public class SinkGraphNode extends AbstractGraphNode { private final Supplier> supplier; private final String topicName; private final String parentName; public SinkGraphNode(String name, String parentName, String topicName, Supplier> supplier) { super(name); this.topicName = topicName; this.supplier = supplier; this.parentName = parentName; } @Override public void addRealNode(TopologyBuilder builder) { builder.addRealSink(name, parentName, topicName, supplier); } @Override public String toString() { return "SinkGraphNode{" + "name=[" + name + "]}"; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/SourceGraphNode.java ================================================ package org.apache.rocketmq.streams.core.topology.virtual; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import org.apache.rocketmq.streams.core.function.supplier.SourceSupplier; import org.apache.rocketmq.streams.core.running.Processor; import org.apache.rocketmq.streams.core.serialization.KeyValueDeserializer; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import java.util.function.Supplier; public class SourceGraphNode extends AbstractGraphNode { private Supplier> supplier; private String topicName; public SourceGraphNode(String name, String topicName, KeyValueDeserializer deserializer) { super(name); this.topicName = topicName; this.supplier = new SourceSupplier<>(topicName, deserializer); } @Override public void addRealNode(TopologyBuilder builder) { builder.addRealSource(name, topicName, supplier); } @Override public String toString() { return "SourceGraphNode{" + "name=[" + name + "]}"; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/util/ColumnFamilyUtil.java ================================================ package org.apache.rocketmq.streams.core.util; import org.apache.rocketmq.streams.core.common.Constant; import org.rocksdb.*; import java.io.UnsupportedEncodingException; import java.nio.charset.StandardCharsets; import java.util.*; import java.util.function.Predicate; public class ColumnFamilyUtil { public static final String WATERMARK_STATE_CF = "watermark-state"; public static final String WINDOW_STATE_CF = "window-state"; public static final String VALUE_STATE_CF = "value-state"; private enum ColumnFamilyEnum { WATERMARK_STATE_COLUMN_FAMILY(WATERMARK_STATE_CF, str -> { return str.startsWith(Constant.WATERMARK_KEY); }), WINDOW_STATE_COLUMN_FAMILY(WINDOW_STATE_CF, str -> { String[] splits = str.split("&&"); return splits.length == 4; }), VALUE_STATE_COLUMN_FAMILY(VALUE_STATE_CF, str -> true) ; private final String name; private final Predicate keyCheckFunc; ColumnFamilyEnum(String name, Predicate keyCheckFunc) { this.name = name; this.keyCheckFunc = keyCheckFunc; } } private final static Map cFName2CFHandle = new HashMap<>(); public static void createColumnFamilies(RocksDB rocksDB, ColumnFamilyOptions cfOpts) throws RocksDBException, UnsupportedEncodingException { final List cfDescriptors = new ArrayList<>(); for (ColumnFamilyEnum columnFamilyEnum : ColumnFamilyEnum.values()) { cfDescriptors.add(new ColumnFamilyDescriptor(columnFamilyEnum.name.getBytes(StandardCharsets.UTF_8), cfOpts)); } List columnFamilyHandles = rocksDB.createColumnFamilies(cfDescriptors); for (ColumnFamilyHandle columnFamilyHandle : columnFamilyHandles) { cFName2CFHandle.put(new String(columnFamilyHandle.getName(), StandardCharsets.UTF_8), columnFamilyHandle); } } public static String getColumnFamilyByKey(byte[] key) { if (key == null) { return null; } return getColumnFamilyByKey(new String(key, StandardCharsets.UTF_8)); } public static String getColumnFamilyByKey(String key) { for (ColumnFamilyEnum columnFamilyEnum : ColumnFamilyEnum.values()) { if (columnFamilyEnum.keyCheckFunc.test(key)) { return columnFamilyEnum.name; } } return null; } public static ColumnFamilyHandle getColumnFamilyHandleByName(String name) { return cFName2CFHandle.get(name); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/util/OperatorNameMaker.java ================================================ package org.apache.rocketmq.streams.core.util; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import java.util.concurrent.atomic.AtomicInteger; public class OperatorNameMaker { public static final String SOURCE_PREFIX = "ROCKETMQ-SOURCE"; public static final String SHUFFLE_SOURCE_PREFIX = "ROCKETMQ-SHUFFLE-SOURCE"; public static final String SINK_PREFIX = "ROCKETMQ-SINK"; public static final String PRINT_PREFIX = "ROCKETMQ-PRINT"; public static final String SHUFFLE_SINK_PREFIX = "ROCKETMQ-SHUFFLE-SINK"; public static final String MAP_PREFIX = "ROCKETMQ-MAP"; public static final String FLAT_MAP_PREFIX = "ROCKETMQ-FLATMAP"; public static final String FILTER_PREFIX = "ROCKETMQ-FILTER"; public static final String GROUPBY_PREFIX = "ROCKETMQ-GROUPBY"; public static final String COUNT_PREFIX = "ROCKETMQ-COUNT"; public static final String MIN_PREFIX = "ROCKETMQ-MIN"; public static final String MAX_PREFIX = "ROCKETMQ-MAX"; public static final String SUM_PREFIX = "ROCKETMQ-SUM"; public static final String FOR_EACH_PREFIX = "ROCKETMQ-FOREACH"; public static final String WINDOW_PREFIX = "ROCKETMQ-WINDOW"; public static final String ADD_TAG = "ROCKETMQ-ADD-TAG"; public static final String WINDOW_AVG_PREFIX = "ROCKETMQ-WINDOW-AVG"; public static final String AGGREGATE_PREFIX = "ROCKETMQ-AGGREGATE"; public static final String ACCUMULATE_PREFIX = "ROCKETMQ-ACCUMULATE"; public static final String JOIN_WINDOW_PREFIX = "JOIN-WINDOW"; public static final String JOIN_PREFIX = "JOIN"; public static final String pattern = "%s-%s-%s"; private static final ThreadLocal index = ThreadLocal.withInitial(() -> new AtomicInteger(0)); private static int incrementAndGet() { return index.get().incrementAndGet(); } public static String makeName(String prefix, String jobId) { String number = String.format("%05d", incrementAndGet()); return String.format(pattern, jobId, prefix, number); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/util/Pair.java ================================================ /* * 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.core.util; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; public class Pair { private T1 key; private T2 value; @JsonCreator public Pair(@JsonProperty("key") T1 key, @JsonProperty("value") T2 value) { this.key = key; this.value = value; } public T1 getKey() { return key; } public void setKey(T1 key) { this.key = key; } public T2 getValue() { return value; } public void setValue(T2 value) { this.value = value; } @Override public String toString() { return "Pair{" + "object1=" + key + ", object2=" + value + '}'; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/util/RocketMQUtil.java ================================================ /* * 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.core.util; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.constant.PermName; import org.apache.rocketmq.common.protocol.ResponseCode; import org.apache.rocketmq.common.protocol.body.ClusterInfo; import org.apache.rocketmq.common.protocol.route.BrokerData; import org.apache.rocketmq.common.protocol.route.QueueData; import org.apache.rocketmq.common.protocol.route.TopicRouteData; import org.apache.rocketmq.remoting.exception.RemotingException; import org.apache.rocketmq.srvutil.ServerUtil; import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; import org.apache.rocketmq.tools.command.CommandUtil; import org.apache.rocketmq.tools.command.topic.UpdateStaticTopicSubCommand; import org.apache.rocketmq.tools.command.topic.UpdateTopicSubCommand; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Set; public class RocketMQUtil { private static final Logger logger = LoggerFactory.getLogger(RocketMQUtil.class.getName()); private static final List existTopic = new ArrayList<>(); //neither static topic nor compact topic. expansion with source topic. public static void createNormalTopic(DefaultMQAdminExt mqAdmin, String topicName, int totalQueueNum, Set clusters) throws Exception { if (check(mqAdmin, topicName)) { logger.info("topic[{}] already exist.", topicName); return; } if (clusters == null || clusters.size() == 0) { clusters = getCluster(mqAdmin); } for (String cluster : clusters) { Set masterSet = CommandUtil.fetchMasterAddrByClusterName(mqAdmin, cluster); int remainder = totalQueueNum % masterSet.size(); if (remainder != 0) { String temp = String.format("can not create topic:%s, total num=%s, master num=%s", topicName, totalQueueNum, masterSet.size()); logger.warn(temp); } int queueNumInEachBroker = totalQueueNum / masterSet.size(); TopicConfig topicConfig = new TopicConfig(topicName, queueNumInEachBroker, queueNumInEachBroker, PermName.PERM_READ | PermName.PERM_WRITE); if (remainder == 0) { for (String addr : masterSet) { mqAdmin.createAndUpdateTopicConfig(addr, topicConfig); logger.info("create topic to broker:{} cluster:{}, success.", addr, cluster); } } else { String[] masterArray = masterSet.toArray(new String[]{}); topicConfig = new TopicConfig(topicName, queueNumInEachBroker + remainder, queueNumInEachBroker + remainder, PermName.PERM_READ | PermName.PERM_WRITE); mqAdmin.createAndUpdateTopicConfig(masterArray[0], topicConfig); for (int i = 1; i < masterArray.length; i++) { topicConfig = new TopicConfig(topicName, queueNumInEachBroker, queueNumInEachBroker, PermName.PERM_READ | PermName.PERM_WRITE); mqAdmin.createAndUpdateTopicConfig(masterArray[0], topicConfig); } } } } public static void createNormalTopic(DefaultMQAdminExt mqAdmin, String sourceTopic, String stateTopic) throws Exception { //找到brokerAddr TopicRouteData topicRouteData = mqAdmin.examineTopicRouteInfo(sourceTopic); List queueData = topicRouteData.getQueueDatas(); List brokerData = topicRouteData.getBrokerDatas(); HashMap brokerName2MaterBrokerAddr = new HashMap<>(); for (BrokerData broker : brokerData) { String masterBrokerAddr = broker.getBrokerAddrs().get(0L); brokerName2MaterBrokerAddr.put(broker.getBrokerName(), masterBrokerAddr); } for (QueueData queue : queueData) { int readQueueNums = queue.getReadQueueNums(); int writeQueueNums = queue.getWriteQueueNums(); String brokerName = queue.getBrokerName(); TopicConfig topicConfig = new TopicConfig(stateTopic, readQueueNums, writeQueueNums); mqAdmin.createAndUpdateTopicConfig(brokerName2MaterBrokerAddr.get(brokerName), topicConfig); } existTopic.add(stateTopic); } //used in RSQLDB,maybe. public static void createStaticCompactTopic(DefaultMQAdminExt mqAdmin, String topicName, int totalQueueNum, Set clusters) throws Exception { if (check(mqAdmin, topicName)) { logger.info("topic[{}] already exist.", topicName); return; } if (clusters == null || clusters.size() == 0) { clusters = getCluster(mqAdmin); } for (String cluster : clusters) { createStaticTopicWithCommand(topicName, totalQueueNum, new HashSet<>(), cluster, mqAdmin.getNamesrvAddr()); logger.info("【step 1】create static topic:[{}] in cluster:[{}] success, logic queue num:[{}].", topicName, cluster, totalQueueNum); update2CompactTopicWithCommand(topicName, totalQueueNum, cluster, mqAdmin.getNamesrvAddr()); logger.info("【step 2】update static topic to compact topic success. topic:[{}], cluster:[{}]", topicName, cluster); } existTopic.add(topicName); logger.info("create static-compact topic [{}] success, queue num [{}]", topicName, totalQueueNum); } public static void createStaticTopic(DefaultMQAdminExt mqAdmin, String topicName, int queueNum) throws Exception { if (check(mqAdmin, topicName)) { logger.info("topic[{}] already exist.", topicName); return; } Set clusters = getCluster(mqAdmin); for (String cluster : clusters) { createStaticTopicWithCommand(topicName, queueNum, new HashSet<>(), cluster, mqAdmin.getNamesrvAddr()); logger.info("create static topic:[{}] in cluster:[{}] success, logic queue num:[{}].", topicName, cluster, queueNum); } existTopic.add(topicName); } private static void createStaticTopicWithCommand(String topic, int totalQueueNum, Set brokers, String cluster, String nameservers) throws Exception { UpdateStaticTopicSubCommand cmd = new UpdateStaticTopicSubCommand(); Options options = ServerUtil.buildCommandlineOptions(new Options()); String[] args; if (cluster != null) { args = new String[]{ "-c", cluster, "-t", topic, "-qn", String.valueOf(totalQueueNum), "-n", nameservers }; } else { String brokerStr = String.join(",", brokers); args = new String[]{ "-b", brokerStr, "-t", topic, "-qn", String.valueOf(totalQueueNum), "-n", nameservers }; } final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), args, cmd.buildCommandlineOptions(options), new PosixParser()); String namesrvAddr = commandLine.getOptionValue('n'); System.setProperty(MixAll.NAMESRV_ADDR_PROPERTY, namesrvAddr); cmd.execute(commandLine, options, null); } private static void update2CompactTopicWithCommand(String topic, int queueNum, String cluster, String nameservers) throws Exception { UpdateTopicSubCommand command = new UpdateTopicSubCommand(); Options options = ServerUtil.buildCommandlineOptions(new Options()); String[] args = new String[]{ "-c", cluster, "-t", topic, "-r", String.valueOf(queueNum), "-w", String.valueOf(queueNum), "-n", nameservers // todo 发布版本还不支持 // , "-a", "+delete.policy=COMPACTION" }; final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + command.commandName(), args, command.buildCommandlineOptions(options), new PosixParser()); String namesrvAddr = commandLine.getOptionValue('n'); System.setProperty(MixAll.NAMESRV_ADDR_PROPERTY, namesrvAddr); command.execute(commandLine, options, null); } public static Set getCluster(DefaultMQAdminExt mqAdmin) throws Exception { ClusterInfo clusterInfo = mqAdmin.examineBrokerClusterInfo(); return clusterInfo.getClusterAddrTable().keySet(); } private static boolean check(DefaultMQAdminExt mqAdmin, String topicName) { if (existTopic.contains(topicName)) { return true; } try { mqAdmin.examineTopicRouteInfo(topicName); existTopic.add(topicName); return true; } catch (RemotingException | InterruptedException e) { logger.error("examine topic route info error.", e); throw new RuntimeException("examine topic route info error.", e); } catch (MQClientException exception) { if (exception.getResponseCode() == ResponseCode.TOPIC_NOT_EXIST) { logger.info("topic[{}] does not exist, create it.", topicName); } else { throw new RuntimeException(exception); } } return false; } public static boolean checkWhetherExist(String topic) { return existTopic.contains(topic); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/util/Utils.java ================================================ /* * 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.core.util; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RStreamsException; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.text.SimpleDateFormat; import java.util.Date; public class Utils { private static final ObjectMapper objectMapper = new ObjectMapper(); public static final String pattern = "%s@%s@%s"; static { objectMapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES) .disable(DeserializationFeature.FAIL_ON_NULL_FOR_PRIMITIVES) .enable(DeserializationFeature.ACCEPT_EMPTY_STRING_AS_NULL_OBJECT) .enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS) .enable(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN) .setNodeFactory(JsonNodeFactory.withExactBigDecimals(true)); } public static String buildKey(String brokerName, String topic, int queueId) { return String.format(pattern, brokerName, topic, queueId); } public static String buildKey(String key, String... args) { if (StringUtils.isEmpty(key)) { return null; } StringBuilder builder = new StringBuilder(); builder.append(key); if (args == null || args.length == 0) { return builder.toString(); } builder.append(Constant.SPLIT); for (String arg : args) { builder.append(arg); builder.append(Constant.SPLIT); } return builder.substring(0, builder.lastIndexOf(Constant.SPLIT)); } public static String[] split(String source) { return split(source, Constant.SPLIT); } public static String[] split(String source, String split) { if (StringUtils.isEmpty(source) || StringUtils.isEmpty(split)) { return new String[]{}; } return source.split(split); } public static byte[] object2Byte(Object target) throws JsonProcessingException { if (target == null) { return new byte[]{}; } return objectMapper.writeValueAsBytes(target); } public static B byte2Object(byte[] source, Class clazz) throws IOException { if (source == null || source.length == 0 || clazz == null) { return null; } return objectMapper.readValue(source, clazz); } public static SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); public static String format(long timestamp) { Date date = new Date(timestamp); return df.format(date); } public static String toHexString(Object obj) { try { if (obj instanceof byte[]) { return DigestUtils.md5Hex((byte[]) obj); } else if (obj instanceof String) { return DigestUtils.md5Hex((String) obj); } else if (obj instanceof InputStream) { return DigestUtils.md5Hex((InputStream) obj); } else { byte[] bytes = object2Byte(obj); return DigestUtils.md5Hex(bytes); } } catch (Throwable t) { throw new RStreamsException("object to HexString error, object=" + obj, t); } } public static byte[] long2Bytes(long time) { ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES); buffer.putLong(time); return buffer.array(); } public static long bytes2Long(byte[] bytes) { if (bytes == null || bytes.length == 0) { return 0; } ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES); buffer.put(bytes); buffer.flip();//need flip return buffer.getLong(); } public static byte[] watermarkKeyBytes(MessageQueue stateTopicMessageQueue, String watermarkPrefix) { if (stateTopicMessageQueue == null || StringUtils.isBlank(watermarkPrefix)) { throw new IllegalArgumentException(); } String key = Utils.buildKey(watermarkPrefix, stateTopicMessageQueue.getBrokerName(), stateTopicMessageQueue.getTopic(), String.valueOf(stateTopicMessageQueue.getQueueId())); assert key != null; return key.getBytes(StandardCharsets.UTF_8); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/JoinType.java ================================================ /* * 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.core.window; public enum JoinType { INNER_JOIN, LEFT_JOIN } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/StreamType.java ================================================ /* * 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.core.window; public enum StreamType { LEFT_STREAM, RIGHT_STREAM } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/Time.java ================================================ /* * 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.core.window; import java.util.concurrent.TimeUnit; import static io.netty.util.internal.ObjectUtil.checkNotNull; public class Time { private final TimeUnit unit; private final long size; private Time(long size, TimeUnit unit) { this.unit = checkNotNull(unit, "time unit may not be null"); this.size = size; } public TimeUnit getUnit() { return unit; } public long getSize() { return size; } public long toMilliseconds() { return unit.toMillis(size); } public long toMillSecond() { return this.unit.toMillis(this.size); } public static Time of(long size, TimeUnit unit) { return new Time(size, unit); } public static Time milliseconds(long milliseconds) { return of(milliseconds, TimeUnit.MILLISECONDS); } public static Time seconds(long seconds) { return of(seconds, TimeUnit.SECONDS); } public static Time minutes(long minutes) { return of(minutes, TimeUnit.MINUTES); } public static Time hours(long hours) { return of(hours, TimeUnit.HOURS); } public static Time days(long days) { return of(days, TimeUnit.DAYS); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/TimeType.java ================================================ /* * 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.core.window; public enum TimeType { EVENT_TIME, PROCESS_TIME } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/Window.java ================================================ /* * 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.core.window; import org.apache.rocketmq.streams.core.util.Utils; public class Window { private long startTime; private long endTime; public Window(long startTime, long endTime) { this.startTime = startTime; this.endTime = endTime; } public long getStartTime() { return startTime; } public void setStartTime(long startTime) { this.startTime = startTime; } public long getEndTime() { return endTime; } public void setEndTime(long endTime) { this.endTime = endTime; } @Override public String toString() { return "[" + "startTime=" + Utils.format(startTime) + "," + "endTime=" + Utils.format(endTime) + "]"; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/WindowBuilder.java ================================================ /* * 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.core.window; public class WindowBuilder { public static WindowInfo tumblingWindow(Time size) { WindowInfo info = new WindowInfo(); info.setWindowType(WindowInfo.WindowType.TUMBLING_WINDOW); info.setWindowSize(size); info.setWindowSlide(size); return info; } public static WindowInfo slidingWindow(Time size, Time slideStep) { WindowInfo info = new WindowInfo(); info.setWindowType(WindowInfo.WindowType.SLIDING_WINDOW); info.setWindowSize(size); info.setWindowSlide(slideStep); return info; } public static WindowInfo sessionWindow(Time size) { WindowInfo info = new WindowInfo(); info.setWindowType(WindowInfo.WindowType.SESSION_WINDOW); info.setSessionTimeout(size); return info; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/WindowInfo.java ================================================ /* * 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.core.window; public class WindowInfo { private WindowType windowType; private JoinStream joinStream = null; private Time windowSize;//窗口大小 private Time windowSlide;//滑动大小 private Time sessionTimeout; public WindowType getWindowType() { return windowType; } public void setWindowType(WindowType windowType) { this.windowType = windowType; } public Time getWindowSize() { return windowSize; } public void setWindowSize(Time windowSize) { this.windowSize = windowSize; } public Time getWindowSlide() { return windowSlide; } public void setWindowSlide(Time windowSlide) { this.windowSlide = windowSlide; } public Time getSessionTimeout() { return sessionTimeout; } public void setSessionTimeout(Time sessionTimeout) { this.sessionTimeout = sessionTimeout; } public JoinStream getJoinStream() { return joinStream; } public void setJoinStream(JoinStream joinStream) { this.joinStream = joinStream; } public enum WindowType { SLIDING_WINDOW, TUMBLING_WINDOW, SESSION_WINDOW } public static class JoinStream { private JoinType joinType; private StreamType streamType; public JoinStream(JoinType joinType, StreamType streamType) { this.joinType = joinType; this.streamType = streamType; } public JoinType getJoinType() { return joinType; } public void setJoinType(JoinType joinType) { this.joinType = joinType; } public StreamType getStreamType() { return streamType; } public void setStreamType(StreamType streamType) { this.streamType = streamType; } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/WindowKey.java ================================================ /* * 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.core.window; import com.google.common.base.Objects; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.streams.core.util.Utils; import java.nio.charset.StandardCharsets; public class WindowKey { private static final String SPLIT = "&&"; private String operatorName; private Long windowStart; private Long windowEnd; private String key2String; public WindowKey(String operatorName, String key2String, Long windowEnd, Long windowStart) { this.operatorName = operatorName; this.windowStart = windowStart; this.windowEnd = windowEnd; this.key2String = key2String; } public String getOperatorName() { return operatorName; } public void setOperatorName(String operatorName) { this.operatorName = operatorName; } public Long getWindowStart() { return windowStart; } public void setWindowStart(Long windowStart) { this.windowStart = windowStart; } public Long getWindowEnd() { return windowEnd; } public void setWindowEnd(Long windowEnd) { this.windowEnd = windowEnd; } public String getKey2String() { return key2String; } public void setKey2String(String key2String) { this.key2String = key2String; } public String getKeyAndWindow() { StringBuilder builder = new StringBuilder(); builder.append(windowStart) .append(WindowKey.SPLIT) .append(windowEnd) .append(WindowKey.SPLIT) .append(key2String); return builder.toString(); } public static WindowKey byte2WindowKey(byte[] source) { String str = new String(source, StandardCharsets.UTF_8); String[] split = Utils.split(str, WindowKey.SPLIT); return new WindowKey(split[0], split[1], Long.parseLong(split[2]), Long.parseLong(split[3])); } public static byte[] windowKey2Byte(WindowKey windowKey) { if (windowKey == null) { return new byte[0]; } return windowKey.toString().getBytes(StandardCharsets.UTF_8); } @Override public int hashCode() { int total = 0; if (!StringUtils.isBlank(operatorName)) { total += operatorName.hashCode(); } if (!StringUtils.isBlank(key2String)) { total += key2String.hashCode(); } total += windowEnd; total += windowStart; return total; } @Override public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; WindowKey windowKey = (WindowKey) o; return Objects.equal(operatorName, windowKey.operatorName) && Objects.equal(windowStart, windowKey.windowStart) && Objects.equal(windowEnd, windowKey.windowEnd) && Objects.equal(key2String, windowKey.key2String); } @Override public String toString() { StringBuilder builder = new StringBuilder(); builder.append(operatorName) .append(WindowKey.SPLIT) .append(key2String) .append(WindowKey.SPLIT) .append(windowEnd) .append(WindowKey.SPLIT) .append(windowStart) ; return builder.toString(); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/WindowState.java ================================================ /* * 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.core.window; import com.fasterxml.jackson.core.JsonProcessingException; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import org.apache.rocketmq.streams.core.util.Utils; import java.io.Serializable; import java.nio.charset.StandardCharsets; /** * windowState data how to encode KV *
 * +-----------+---------------+-----------+-------------+
 * | Int(4)    | Int(4)        | key bytes | value bytes |
 * | key length| value length  |           |             |
 * +-----------+---------------+-----------+-------------+
 * 
*/ public class WindowState implements Serializable { private static final long serialVersionUID = 1669344441528746814L; private long recordEarliestTimestamp = Long.MAX_VALUE; private long recordLastTimestamp; private K key; private V value; private byte[] keyBytes; private byte[] valueBytes; private Class keyClazz; private Class valueClazz; //only for Serializer/Deserializer public WindowState() { } public WindowState(K key, V value, long recordLastTimestamp) throws JsonProcessingException { this.key = key; this.value = value; this.recordLastTimestamp = recordLastTimestamp; if (key != null) { this.keyBytes = Utils.object2Byte(key); this.keyClazz = key.getClass(); } if (value != null) { this.valueBytes = Utils.object2Byte(value); this.valueClazz = value.getClass(); } } public K getKey() { return key; } public void setKey(K key) throws JsonProcessingException { this.key = key; this.keyBytes = Utils.object2Byte(key); if (key != null) { this.keyClazz = key.getClass(); } } public V getValue() { return value; } public void setValue(V value) throws JsonProcessingException { this.value = value; this.valueBytes = Utils.object2Byte(value); if (value != null) { this.valueClazz = value.getClass(); } } public byte[] getKeyBytes() { return keyBytes; } public void setKeyBytes(byte[] keyBytes) { this.keyBytes = keyBytes; } public byte[] getValueBytes() { return valueBytes; } public void setValueBytes(byte[] valueBytes) { this.valueBytes = valueBytes; } @SuppressWarnings("unchecked") public Class getValueClazz() { return (Class) valueClazz; } public void setValueClazz(Class valueClazz) { this.valueClazz = valueClazz; } public void setKeyClazz(Class keyClazz) { this.keyClazz = keyClazz; } @SuppressWarnings("unchecked") public Class getKeyClazz() { return (Class) keyClazz; } public long getRecordEarliestTimestamp() { return recordEarliestTimestamp; } public void setRecordEarliestTimestamp(long recordEarliestTimestamp) { this.recordEarliestTimestamp = recordEarliestTimestamp; } public long getRecordLastTimestamp() { return recordLastTimestamp; } public void setRecordLastTimestamp(long recordLastTimestamp) { this.recordLastTimestamp = recordLastTimestamp; } private static final ByteBuf buf = Unpooled.buffer(16); public static byte[] windowState2Byte(WindowState state) throws Throwable { if (state == null) { return new byte[0]; } Class keyClazz = state.getKeyClazz(); if (keyClazz == null) { keyClazz = state.getKey().getClass(); } byte[] keyClazzBytes = keyClazz.getName().getBytes(StandardCharsets.UTF_8); byte[] keyBytes = state.getKeyBytes(); if (keyBytes == null) { keyBytes = Utils.object2Byte(state.getKey()); } Class valueClazz = state.getValueClazz(); if (valueClazz == null) { valueClazz = state.getValue().getClass(); } byte[] valueClazzBytes = valueClazz.getName().getBytes(StandardCharsets.UTF_8); byte[] valueBytes = state.getValueBytes(); if (valueBytes == null) { valueBytes = Utils.object2Byte(state.getValue()); } int length = 4 + 8 + 8 + 4 + keyClazzBytes.length + 4 + keyBytes.length + 4 + valueClazzBytes.length + 4 + valueBytes.length; buf.writeInt(length); buf.writeLong(state.getRecordLastTimestamp()); buf.writeLong(state.getRecordEarliestTimestamp()); //key class buf.writeInt(keyClazzBytes.length); buf.writeBytes(keyClazzBytes); //key buf.writeInt(keyBytes.length); buf.writeBytes(keyBytes); //value class buf.writeInt(valueClazzBytes.length); buf.writeBytes(valueClazzBytes); //value buf.writeInt(valueBytes.length); buf.writeBytes(valueBytes); byte[] bytes = new byte[buf.readableBytes()]; buf.readBytes(bytes); buf.clear(); return bytes; } public static WindowState byte2WindowState(byte[] bytes) throws Throwable { ByteBuf byteBuf = Unpooled.wrappedBuffer(bytes); int totalLength = byteBuf.readInt(); if (bytes.length < totalLength) { //上层已经拆好了包 throw new IllegalArgumentException("byteBuf length less than total"); } long recordLastTimestamp = byteBuf.readLong(); long recordEarliestTimestamp = byteBuf.readLong(); //key class int keyClazzLength = byteBuf.readInt(); ByteBuf buf = byteBuf.readBytes(keyClazzLength); byte[] keyClazzBytes = new byte[keyClazzLength]; buf.readBytes(keyClazzBytes); //实例化 String keyClassName = new String(keyClazzBytes, StandardCharsets.UTF_8); Class keyClazz = Class.forName(keyClassName); //key int keyLength = byteBuf.readInt(); ByteBuf keyBuf = byteBuf.readBytes(keyLength); byte[] keyBytes = new byte[keyLength]; keyBuf.readBytes(keyBytes); //value class int valueClazzLength = byteBuf.readInt(); ByteBuf valueClazzBuf = byteBuf.readBytes(valueClazzLength); byte[] valueClazzBytes = new byte[valueClazzLength]; valueClazzBuf.readBytes(valueClazzBytes); //实例化 String valueClassName = new String(valueClazzBytes, StandardCharsets.UTF_8); Class valueClazz = Class.forName(valueClassName); //value int valueLength = byteBuf.readInt(); ByteBuf valueBuf = byteBuf.readBytes(valueLength); byte[] valueBytes = new byte[valueLength]; valueBuf.readBytes(valueBytes); WindowState result = new WindowState<>(); result.setRecordLastTimestamp(recordLastTimestamp); result.setRecordEarliestTimestamp(recordEarliestTimestamp); result.setKeyBytes(keyBytes); result.setValueBytes(valueBytes); result.setKeyClazz(keyClazz); result.setValueClazz(valueClazz); result.setKey(Utils.byte2Object(keyBytes, result.getKeyClazz())); result.setValue(Utils.byte2Object(valueBytes, result.getValueClazz())); byteBuf.release(); buf.release(); keyBuf.release(); valueBuf.release(); valueClazzBuf.release(); return result; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/WindowStore.java ================================================ /* * 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.core.window; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.state.StateStore; import org.apache.rocketmq.streams.core.util.ColumnFamilyUtil; import org.apache.rocketmq.streams.core.util.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; public class WindowStore { private static final Logger logger = LoggerFactory.getLogger(WindowStore.class.getName()); private StateStore stateStore; private ValueMapperAction> bytes2State; private ValueMapperAction, byte[]> state2Bytes; public WindowStore(StateStore stateStore, ValueMapperAction> bytes2State, ValueMapperAction, byte[]> state2Bytes) { this.stateStore = stateStore; this.bytes2State = bytes2State; this.state2Bytes = state2Bytes; } public void put(MessageQueue stateTopicMessageQueue, WindowKey windowKey, WindowState value) throws Throwable { logger.debug("put key into store, key: " + windowKey); byte[] keyBytes = WindowKey.windowKey2Byte(windowKey); byte[] valueBytes = this.state2Bytes.convert(value); this.stateStore.put(stateTopicMessageQueue, ColumnFamilyUtil.WINDOW_STATE_CF, keyBytes, valueBytes); } public WindowState get(WindowKey windowKey) throws Throwable { byte[] bytes = WindowKey.windowKey2Byte(windowKey); byte[] valueBytes = this.stateStore.get(ColumnFamilyUtil.WINDOW_STATE_CF, bytes); return deserializerState(valueBytes); } public List>> searchLessThanWatermark(String operatorName, long lessThanThisTime) throws Throwable { List> windowStateBytes = this.stateStore.searchStateLessThanWatermark(operatorName, lessThanThisTime, WindowKey::byte2WindowKey); return deserializerState(windowStateBytes); } public List>> searchMatchKeyPrefix(String operatorName) throws Throwable { List> pairs = this.stateStore.searchStateLessThanWatermark(operatorName, Long.MAX_VALUE, WindowKey::byte2WindowKey); return deserializerState(pairs); } public void deleteByKey(WindowKey windowKey) throws Throwable { if (windowKey == null) { return; } byte[] keyBytes = WindowKey.windowKey2Byte(windowKey); this.stateStore.delete(keyBytes); } private List>> deserializerState(List> windowStateBytes) throws Throwable { List>> result = new ArrayList<>(); if (windowStateBytes == null || windowStateBytes.size() == 0) { return result; } for (Pair pair : windowStateBytes) { byte[] keyBytes = pair.getKey(); WindowKey key = WindowKey.byte2WindowKey(keyBytes); WindowState state = this.deserializerState(pair.getValue()); Pair> temp = new Pair<>(key, state); result.add(temp); } return result; } private WindowState deserializerState(byte[] source) throws Throwable { if (source == null) { return null; } return this.bytes2State.convert(source); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AbstractWindowFire.java ================================================ /* * 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.core.window.fire; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.state.StateStore; import org.apache.rocketmq.streams.core.util.Utils; import java.util.HashSet; import java.util.Set; import java.util.function.BiFunction; public abstract class AbstractWindowFire implements WindowFire { protected final StreamContext context; private final MessageQueue stateTopicMessageQueue; private final BiFunction commitWatermark; public AbstractWindowFire(StreamContext context, MessageQueue stateTopicMessageQueue, BiFunction commitWatermark) { this.context = context; this.stateTopicMessageQueue = stateTopicMessageQueue; this.commitWatermark = commitWatermark; } void commitWatermark(long watermark) { this.commitWatermark.apply(watermark, stateTopicMessageQueue); } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AccumulatorSessionWindowFire.java ================================================ /* * 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.core.window.fire; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RStreamsException; import org.apache.rocketmq.streams.core.function.accumulator.Accumulator; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.window.WindowState; import org.apache.rocketmq.streams.core.window.WindowStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Properties; import java.util.function.BiFunction; public class AccumulatorSessionWindowFire extends AbstractWindowFire { private static final Logger logger = LoggerFactory.getLogger(AccumulatorSessionWindowFire.class); private final WindowStore> windowStore; public AccumulatorSessionWindowFire(WindowStore> windowStore, StreamContext context, MessageQueue stateTopicMessageQueue, BiFunction commitWatermark) { super(context, stateTopicMessageQueue, commitWatermark); this.windowStore = windowStore; } public List fire(String operatorName, long watermark) { List fired = new ArrayList<>(); try { List>>> pairs = windowStore.searchLessThanWatermark(operatorName, watermark); Iterator>>> iterator = pairs.iterator(); while (iterator.hasNext()) { Pair>> pair = iterator.next(); WindowKey windowKey = pair.getKey(); WindowState> state = pair.getValue(); long windowEnd = windowKey.getWindowEnd(); long windowBegin; if (state.getRecordEarliestTimestamp() == Long.MAX_VALUE) { windowBegin = windowKey.getWindowStart(); } else { windowBegin = state.getRecordEarliestTimestamp(); } logger.info("fire session,windowKey={}, search keyPrefix={}, window: [{} - {}]", windowKey, state.getKey().toString(), Utils.format(windowBegin), Utils.format(windowEnd)); Properties header = context.getHeader(); header.put(Constant.WINDOW_START_TIME, windowBegin); header.put(Constant.WINDOW_END_TIME, windowEnd); Accumulator value = state.getValue(); OV data = value.result(header); Data result = new Data<>(state.getKey(), data, state.getRecordLastTimestamp(), header); Data convert = this.convert(result); context.forward(convert); //删除状态 windowStore.deleteByKey(windowKey); fired.add(windowKey); } return fired; } catch (Throwable t) { String format = String.format("fire session window error, name:%s", operatorName); throw new RStreamsException(format, t); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AccumulatorWindowFire.java ================================================ /* * 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.core.window.fire; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RStreamsException; import org.apache.rocketmq.streams.core.function.accumulator.Accumulator; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.window.WindowState; import org.apache.rocketmq.streams.core.window.WindowStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; import java.util.Properties; import java.util.function.BiFunction; public class AccumulatorWindowFire extends AbstractWindowFire { private static final Logger logger = LoggerFactory.getLogger(AccumulatorWindowFire.class); private final WindowStore> windowStore; public AccumulatorWindowFire(WindowStore> windowStore, StreamContext context, MessageQueue stateTopicMessageQueue, BiFunction commitWatermark) { super(context, stateTopicMessageQueue, commitWatermark); this.windowStore = windowStore; } public List fire(String operatorName, long watermark) { List fired = new ArrayList<>(); try { List>>> pairs = windowStore.searchLessThanWatermark(operatorName, watermark); //pairs中最后一个时间最小,应该最先触发 for (int i = pairs.size() - 1; i >= 0; i--) { Pair>> pair = pairs.get(i); WindowKey windowKey = pair.getKey(); WindowState> value = pair.getValue(); Long windowEnd = windowKey.getWindowEnd(); Properties header = context.getHeader(); header.put(Constant.WINDOW_START_TIME, windowKey.getWindowStart()); header.put(Constant.WINDOW_END_TIME, windowEnd); Accumulator rovAccumulator = value.getValue(); OV data = rovAccumulator.result(header); Data result = new Data<>(value.getKey(), data, value.getRecordLastTimestamp(), header); Data convert = this.convert(result); if (logger.isDebugEnabled()) { logger.debug("fire window, windowKey={}, search watermark={}, window: [{} - {}], data to next:[{}]", windowKey, watermark, Utils.format(windowKey.getWindowStart()), Utils.format(windowEnd), convert); } context.forward(convert); //删除状态 windowStore.deleteByKey(windowKey); fired.add(windowKey); } } catch (Throwable t) { String format = String.format("fire window error, watermark:%s, operatorName:%s", watermark, operatorName); throw new RStreamsException(format, t); } return fired; } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AggregateSessionWindowFire.java ================================================ /* * 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.core.window.fire; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RStreamsException; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.window.WindowState; import org.apache.rocketmq.streams.core.window.WindowStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Properties; import java.util.function.BiFunction; public class AggregateSessionWindowFire extends AbstractWindowFire { private static final Logger logger = LoggerFactory.getLogger(AggregateWindowFire.class); private final WindowStore windowStore; public AggregateSessionWindowFire(WindowStore windowStore, MessageQueue stateTopicMessageQueue, StreamContext context, BiFunction commitWatermark) { super(context, stateTopicMessageQueue, commitWatermark); this.windowStore = windowStore; } @Override public List fire(String operatorName, long watermark) { List fired = new ArrayList<>(); try { List>> pairs = this.windowStore.searchLessThanWatermark(operatorName, watermark); Iterator>> iterator = pairs.iterator(); while (iterator.hasNext()) { Pair> pair = iterator.next(); WindowKey windowKey = pair.getKey(); WindowState state = pair.getValue(); long windowEnd = windowKey.getWindowEnd(); long windowBegin; if (state.getRecordEarliestTimestamp() == Long.MAX_VALUE) { windowBegin = windowKey.getWindowStart(); } else { windowBegin = state.getRecordEarliestTimestamp(); } logger.info("fire session,windowKey={}, search keyPrefix={}, window: [{} - {}]", windowKey, state.getKey().toString(), Utils.format(windowBegin), Utils.format(windowEnd)); Properties header = this.context.getHeader(); header.put(Constant.WINDOW_START_TIME, windowBegin); header.put(Constant.WINDOW_END_TIME, windowEnd); Data result = new Data<>(state.getKey(), state.getValue(), state.getRecordLastTimestamp(), header); Data convert = this.convert(result); this.context.forward(convert); //删除状态 this.windowStore.deleteByKey(windowKey); fired.add(windowKey); } return fired; } catch (Throwable t) { String format = String.format("fire session window error, name:%s", operatorName); throw new RStreamsException(format, t); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AggregateWindowFire.java ================================================ /* * 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.core.window.fire; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RStreamsException; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.window.WindowState; import org.apache.rocketmq.streams.core.window.WindowStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; import java.util.Properties; import java.util.function.BiFunction; public class AggregateWindowFire extends AbstractWindowFire { private static final Logger logger = LoggerFactory.getLogger(AggregateWindowFire.class); private final WindowStore windowStore; public AggregateWindowFire(WindowStore windowStore, MessageQueue stateTopicMessageQueue, StreamContext context, BiFunction commitWatermark) { super(context, stateTopicMessageQueue, commitWatermark); this.windowStore = windowStore; } @Override public List fire(String operatorName, long watermark) { List fired = new ArrayList<>(); try { List>> pairs = this.windowStore.searchLessThanWatermark(operatorName, watermark); //pairs中最后一个时间最小,应该最先触发 for (int i = pairs.size() - 1; i >= 0; i--) { Pair> pair = pairs.get(i); WindowKey windowKey = pair.getKey(); WindowState value = pair.getValue(); Long windowEnd = windowKey.getWindowEnd(); Properties header = this.context.getHeader(); header.put(Constant.WINDOW_START_TIME, windowKey.getWindowStart()); header.put(Constant.WINDOW_END_TIME, windowEnd); Data result = new Data<>(value.getKey(), value.getValue(), value.getRecordLastTimestamp(), header); Data convert = this.convert(result); if (logger.isDebugEnabled()) { logger.debug("fire window, windowKey={}, search watermark={}, window: [{} - {}], data to next:[{}]", windowKey, watermark, Utils.format(windowKey.getWindowStart()), Utils.format(windowEnd), convert); } this.context.forward(convert); //删除状态 this.windowStore.deleteByKey(windowKey); fired.add(windowKey); } return fired; } catch (Throwable t) { String format = String.format("fire window error, watermark:%s, operatorName:%s", watermark, operatorName); throw new RStreamsException(format, t); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/IdleWindowScaner.java ================================================ /* * 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.core.window.fire; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.window.StreamType; import org.apache.rocketmq.streams.core.window.WindowKey; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; public class IdleWindowScaner implements AutoCloseable { private static final Logger logger = LoggerFactory.getLogger(IdleWindowScaner.class.getName()); private final Integer maxIdleTime; private final ScheduledExecutorService executor; private final ConcurrentHashMap lastUpdateTime2WindowKey = new ConcurrentHashMap<>(16); private final ConcurrentHashMap> fireWindowCallBack = new ConcurrentHashMap<>(16); private final ConcurrentHashMap> fireSessionWindowCallback = new ConcurrentHashMap<>(16); private final ConcurrentHashMap> windowKeyAggregate = new ConcurrentHashMap<>(16); private final ConcurrentHashMap> windowKeyAggregateSession = new ConcurrentHashMap<>(16); private final ConcurrentHashMap> fireJoinWindowCallback = new ConcurrentHashMap<>(16); public IdleWindowScaner(Integer maxIdleTime, ScheduledExecutorService executor) { this.maxIdleTime = maxIdleTime; this.executor = executor; this.executor.scheduleAtFixedRate(() -> { try { scanAndFireWindow(); } catch (Throwable t) { logger.error("scan and fire the idle window error.", t); } }, 0, 1000, TimeUnit.MILLISECONDS); } public void putAccumulatorWindowCallback(WindowKey windowKey, long watermark, AccumulatorWindowFire function) { this.fireWindowCallBack.putIfAbsent(windowKey, function); this.lastUpdateTime2WindowKey.compute(windowKey, (key, timeType) -> { if (timeType == null) { timeType = new TimeType(Type.AccumulatorWindow, System.currentTimeMillis(), watermark); } else { timeType.setUpdateTime(System.currentTimeMillis()); timeType.setWatermark(watermark); } return timeType; }); } public void putAccumulatorSessionWindowCallback(WindowKey windowKey, long watermark, AccumulatorSessionWindowFire function) { this.fireSessionWindowCallback.putIfAbsent(windowKey, function); this.lastUpdateTime2WindowKey.compute(windowKey, (key, timeType) -> { if (timeType == null) { timeType = new TimeType(Type.AccumulatorSessionWindow, System.currentTimeMillis(), watermark); } else { timeType.setUpdateTime(System.currentTimeMillis()); timeType.setWatermark(watermark); } return timeType; }); } public void putAggregateWindowCallback(WindowKey windowKey, long watermark, AggregateWindowFire function) { this.windowKeyAggregate.putIfAbsent(windowKey, function); this.lastUpdateTime2WindowKey.compute(windowKey, (key, timeType) -> { if (timeType == null) { timeType = new TimeType(Type.AggregateWindow, System.currentTimeMillis(), watermark); } else { timeType.setUpdateTime(System.currentTimeMillis()); timeType.setWatermark(watermark); } return timeType; }); } public void putAggregateSessionWindowCallback(WindowKey windowKey, long watermark, AggregateSessionWindowFire function) { this.windowKeyAggregateSession.putIfAbsent(windowKey, function); this.lastUpdateTime2WindowKey.compute(windowKey, (key, timeType) -> { if (timeType == null) { timeType = new TimeType(Type.AggregateSessionWindow, System.currentTimeMillis(), watermark); } else { timeType.setUpdateTime(System.currentTimeMillis()); timeType.setWatermark(watermark); } return timeType; }); } public void putJoinWindowCallback(WindowKey windowKey, long watermark, JoinWindowFire function) { this.fireJoinWindowCallback.putIfAbsent(windowKey, function); this.lastUpdateTime2WindowKey.compute(windowKey, (key, timeType) -> { if (timeType == null) { timeType = new TimeType(Type.JoinWindow, System.currentTimeMillis(), watermark); } else { timeType.setUpdateTime(System.currentTimeMillis()); timeType.setWatermark(watermark); } return timeType; }); } public void removeOldAccumulatorSession(WindowKey oldWindowKey) { if (oldWindowKey == null) { return; } TimeType timeType = this.lastUpdateTime2WindowKey.get(oldWindowKey); if (timeType != null && timeType.getType() == Type.AccumulatorSessionWindow) { this.lastUpdateTime2WindowKey.remove(oldWindowKey); } this.fireSessionWindowCallback.remove(oldWindowKey); } public void removeOldAggregateSession(WindowKey oldWindowKey) { TimeType timeType = this.lastUpdateTime2WindowKey.get(oldWindowKey); if (timeType != null && timeType.getType() == Type.AggregateSessionWindow) { this.lastUpdateTime2WindowKey.remove(oldWindowKey); } this.windowKeyAggregateSession.remove(oldWindowKey); } public void removeWindowKey(WindowKey windowKey) { lastUpdateTime2WindowKey.remove(windowKey); fireWindowCallBack.remove(windowKey); fireSessionWindowCallback.remove(windowKey); windowKeyAggregate.remove(windowKey); windowKeyAggregateSession.remove(windowKey); fireJoinWindowCallback.remove(windowKey); } private void scanAndFireWindow() throws Throwable { Iterator> iterator = this.lastUpdateTime2WindowKey.entrySet().iterator(); while (iterator.hasNext()) { Map.Entry next = iterator.next(); WindowKey windowKey = next.getKey(); TimeType timeType = next.getValue(); Type type = timeType.getType(); long updateTime = timeType.getUpdateTime(); long idleTime = System.currentTimeMillis() - updateTime; switch (type) { case AggregateSessionWindow: case AccumulatorSessionWindow: { long watermark = timeType.getWatermark() + idleTime; if (watermark > windowKey.getWindowEnd()) { try { doFire(windowKey, type, watermark); } finally { iterator.remove(); } } break; } case AccumulatorWindow: case JoinWindow: case AggregateWindow: { long watermark = timeType.getWatermark() + idleTime; if (idleTime > this.maxIdleTime && watermark > windowKey.getWindowEnd()) { try { doFire(windowKey, type, watermark); } finally { iterator.remove(); } } break; } default: throw new UnsupportedOperationException("unknown window type: " + type); } } } private void doFire(WindowKey windowKey, Type type, long watermark) throws Throwable { String operatorName = windowKey.getOperatorName(); switch (type) { case AccumulatorWindow: { AccumulatorWindowFire func = this.fireWindowCallBack.remove(windowKey); if (func != null) { //write the result out, delete the state from local and remote func.fire(operatorName, watermark); //commit watermark to local and remote. func.commitWatermark(watermark); } break; } case AccumulatorSessionWindow: { AccumulatorSessionWindowFire accumulatorSessionWindowFire = this.fireSessionWindowCallback.remove(windowKey); if (accumulatorSessionWindowFire != null) { accumulatorSessionWindowFire.fire(operatorName, watermark); accumulatorSessionWindowFire.commitWatermark(watermark); } break; } case AggregateWindow: { AggregateWindowFire aggregateWindowFire = this.windowKeyAggregate.remove(windowKey); if (aggregateWindowFire != null) { aggregateWindowFire.fire(operatorName, watermark); aggregateWindowFire.commitWatermark(watermark); } break; } case AggregateSessionWindow: { AggregateSessionWindowFire sessionWindowFire = this.windowKeyAggregateSession.remove(windowKey); if (sessionWindowFire != null) { sessionWindowFire.fire(operatorName, watermark); sessionWindowFire.commitWatermark(watermark); } break; } case JoinWindow: { JoinWindowFire joinWindowFire = this.fireJoinWindowCallback.remove(windowKey); if (joinWindowFire != null) { String name = operatorName.substring(0, operatorName.lastIndexOf(Constant.SPLIT)); String streamType = operatorName.substring(operatorName.lastIndexOf(Constant.SPLIT) + 1); joinWindowFire.fire(name, watermark, StreamType.valueOf(streamType)); joinWindowFire.commitWatermark(watermark); } break; } } } @Override public void close() throws Exception { this.executor.shutdown(); } static class TimeType { private Type type; private long updateTime; private long watermark; public TimeType(Type type, long updateTime, long watermark) { this.type = type; this.updateTime = updateTime; this.watermark = watermark; } public Type getType() { return type; } public void setType(Type type) { this.type = type; } public long getUpdateTime() { return updateTime; } public void setUpdateTime(long updateTime) { this.updateTime = updateTime; } public long getWatermark() { return watermark; } public void setWatermark(long watermark) { this.watermark = watermark; } } enum Type { AccumulatorWindow, AccumulatorSessionWindow, AggregateWindow, AggregateSessionWindow, JoinWindow } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/JoinWindowFire.java ================================================ /* * 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.core.window.fire; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.streams.core.common.Constant; import org.apache.rocketmq.streams.core.exception.RStreamsException; import org.apache.rocketmq.streams.core.function.ValueJoinAction; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.running.StreamContext; import org.apache.rocketmq.streams.core.state.StateStore; import org.apache.rocketmq.streams.core.util.ColumnFamilyUtil; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.Utils; import org.apache.rocketmq.streams.core.window.JoinType; import org.apache.rocketmq.streams.core.window.StreamType; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.window.WindowState; import org.apache.rocketmq.streams.core.window.WindowStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Properties; import java.util.Set; import java.util.function.BiFunction; public class JoinWindowFire { private static final Logger logger = LoggerFactory.getLogger(JoinWindowFire.class); private final JoinType joinType; private final MessageQueue stateTopicMessageQueue; private final StreamContext context; private final ValueJoinAction joinAction; private final WindowStore leftWindowStore; private final WindowStore rightWindowStore; private final BiFunction commitWatermark; public JoinWindowFire(JoinType joinType, MessageQueue stateTopicMessageQueue, StreamContext context, ValueJoinAction joinAction, WindowStore leftWindowStore, WindowStore rightWindowStore, BiFunction commitWatermark) { this.joinType = joinType; this.stateTopicMessageQueue = stateTopicMessageQueue; this.context = context; this.joinAction = joinAction; this.leftWindowStore = leftWindowStore; this.rightWindowStore = rightWindowStore; this.commitWatermark = commitWatermark; } public List fire(String operatorName, long watermark, StreamType streamType) { List fired = new ArrayList<>(); try { String leftWindow = Utils.buildKey(operatorName, StreamType.LEFT_STREAM.name()); List>> leftPairs = this.leftWindowStore.searchLessThanWatermark(leftWindow, watermark); if (leftPairs.size() != 0) { for (Pair> leftPair : leftPairs) { logger.debug("search with key prefix:{} and watermark:{}, find window: {}", leftWindow, Utils.format(watermark), leftPair.getKey()); } } String rightWindow = Utils.buildKey(operatorName, StreamType.RIGHT_STREAM.name()); List>> rightPairs = this.rightWindowStore.searchLessThanWatermark(rightWindow, watermark); if (rightPairs.size() != 0) { for (Pair> rightPair : rightPairs) { logger.debug("search with key prefix:{} and watermark:{}, find window: {}", rightWindow, Utils.format(watermark), rightPair.getKey()); } } if (leftPairs.size() == 0 && rightPairs.size() == 0) { logger.debug("left window and right window are all empty, watermark:{}." + "left window operatorName:{}, right window operatorName:{}", Utils.format(watermark), leftWindow, rightWindow); return fired; } leftPairs.sort(Comparator.comparing(pair -> { WindowKey key = pair.getKey(); return key.getWindowEnd(); })); rightPairs.sort(Comparator.comparing(pair -> { WindowKey key = pair.getKey(); return key.getWindowEnd(); })); switch (joinType) { case INNER_JOIN: //匹配上才触发 for (Pair> leftPair : leftPairs) { WindowKey leftWindowKey = leftPair.getKey(); String leftPrefix = leftWindowKey.getKeyAndWindow(); for (Pair> rightPair : rightPairs) { String rightPrefix = rightPair.getKey().getKeyAndWindow(); //相同window中相同key,聚合 if (leftPrefix.equals(rightPrefix)) { //do fire V1 o1 = leftPair.getValue().getValue(); V2 o2 = rightPair.getValue().getValue(); OUT out = this.joinAction.apply(o1, o2); Properties header = this.context.getHeader(); header.put(Constant.WINDOW_START_TIME, leftWindowKey.getWindowStart()); header.put(Constant.WINDOW_END_TIME, leftWindowKey.getWindowEnd()); assert leftPair.getValue().getKey() == rightPair.getValue().getKey(); Data result = new Data<>(leftPair.getValue().getKey(), out, this.context.getDataTime(), header); Data convert = this.convert(result); this.context.forward(convert); fired.add(leftWindowKey); } } } break; case LEFT_JOIN: switch (streamType) { case LEFT_STREAM: //左流全部触发,不管右流匹配上没 for (Pair> leftPair : leftPairs) { WindowKey leftWindowKey = leftPair.getKey(); fired.add(leftWindowKey); String leftPrefix = leftWindowKey.getKeyAndWindow(); Pair> targetPair = null; for (Pair> rightPair : rightPairs) { if (rightPair.getKey().getKeyAndWindow().equals(leftPrefix)) { targetPair = rightPair; break; } } //fire V1 o1 = leftPair.getValue().getValue(); V2 o2 = null; if (targetPair != null) { o2 = targetPair.getValue().getValue(); fired.add(targetPair.getKey()); assert leftPair.getValue().getKey() == targetPair.getValue().getKey(); } OUT out = this.joinAction.apply(o1, o2); Properties header = this.context.getHeader(); header.put(Constant.WINDOW_START_TIME, leftWindowKey.getWindowStart()); header.put(Constant.WINDOW_END_TIME, leftWindowKey.getWindowEnd()); Data result = new Data<>(leftPair.getValue().getKey(), out, this.context.getDataTime(), header); Data convert = this.convert(result); this.context.forward(convert); } break; case RIGHT_STREAM: //do nothing. } break; } if (leftPairs.size() != 0) { logger.debug("delete left window."); for (Pair> leftPair : leftPairs) { this.leftWindowStore.deleteByKey(leftPair.getKey()); } } if (rightPairs.size() != 0) { logger.debug("delete right window."); for (Pair> rightPair : rightPairs) { this.rightWindowStore.deleteByKey(rightPair.getKey()); } } } catch (Throwable t) { String format = String.format("fire window error, watermark:%s.", watermark); throw new RStreamsException(format, t); } return fired; } @SuppressWarnings("unchecked") private Data convert(Data data) { return (Data) new Data<>(data.getKey(), data.getValue(), data.getTimestamp(), data.getHeader()); } void commitWatermark(long watermark) throws Throwable { StateStore stateStore = this.context.getStateStore(); //get old watermark byte[] keyBytes = Utils.watermarkKeyBytes(stateTopicMessageQueue, Constant.WATERMARK_KEY); byte[] watermarkBytes = stateStore.get(ColumnFamilyUtil.WATERMARK_STATE_CF, keyBytes); long oldWatermark = Utils.bytes2Long(watermarkBytes); if (watermark > oldWatermark) { this.commitWatermark.apply(watermark, stateTopicMessageQueue); Set set = new HashSet<>(); set.add(stateTopicMessageQueue); stateStore.persist(set); } } } ================================================ FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/WindowFire.java ================================================ /* * 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.core.window.fire; import org.apache.rocketmq.streams.core.metadata.Data; import org.apache.rocketmq.streams.core.window.WindowKey; import java.util.List; public interface WindowFire { List fire(String operatorName, long watermark); @SuppressWarnings("unchecked") default Data convert(Data data) { return (Data) new Data<>(data.getKey(), data.getValue(), data.getTimestamp(), data.getHeader()); } } ================================================ FILE: core/src/main/resources/logback.xml ================================================ ${user.home}/logs/rocketmq-streams/streams_default.log true ${user.home}/logs/rocketmq-streams/otherdays/streams_default.%i.log 1 10 100MB %d{yyy-MM-dd HH:mm:ss,GMT+8} %p - %m%n UTF-8 %d{yyyy-MM-dd HH:mm:ss.SSS} [%thread] %-5level %logger{50} - %msg%n UTF-8 ================================================ FILE: core/src/test/java/org/apache/rocketmq/streams/core/Num.java ================================================ /* * 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.core; public class Num { private Integer number; public Integer getNumber() { return number; } public void setNumber(Integer number) { this.number = number; } } ================================================ FILE: core/src/test/java/org/apache/rocketmq/streams/core/Test.java ================================================ /* * 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.core; public class Test { } ================================================ FILE: core/src/test/java/org/apache/rocketmq/streams/core/User.java ================================================ /* * 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.core; public class User { private String name; public String getName() { return name; } public void setName(String name) { this.name = name; } } ================================================ FILE: core/src/test/java/org/apache/rocketmq/streams/core/function/AggregateSupplier.java ================================================ /* * 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.core.function; import com.fasterxml.jackson.core.JsonProcessingException; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import org.apache.rocketmq.streams.core.window.WindowState; import org.apache.rocketmq.streams.core.Num; import org.apache.rocketmq.streams.core.User; import org.apache.rocketmq.streams.core.util.Utils; import java.nio.charset.StandardCharsets; public class AggregateSupplier { public static void main(String[] args) throws Throwable { WindowState state = new WindowState<>(); Num num = new Num(); num.setNumber(10); User user = new User(); user.setName("zeni"); state.setKey(num); state.setValue(user); byte[] bytes = object2Byte(state); WindowState result = byte2Object(bytes); System.out.println(result); } private static final ByteBuf buf = Unpooled.buffer(16); protected static byte[] object2Byte(Object obj) throws JsonProcessingException { if (obj == null) { return new byte[]{}; } String name = obj.getClass().getName(); byte[] className = name.getBytes(StandardCharsets.UTF_8); byte[] objBytes = Utils.object2Byte(obj); buf.writeInt(className.length); buf.writeBytes(className); buf.writeInt(objBytes.length); buf.writeBytes(objBytes); byte[] bytes = new byte[buf.readableBytes()]; buf.readBytes(bytes); buf.clear(); buf.release(); return bytes; } /** * decode *
     * +-----------+---------------+-------------+-------------+
     * | Int(4)    |   className  | Int(4)       | value bytes |
     * | classname |              |object length |             |
     * +-----------+--------------+---------------+-------------+
     * 
*/ @SuppressWarnings("unchecked") public static V byte2Object(byte[] bytes) throws Throwable { ByteBuf byteBuf = Unpooled.wrappedBuffer(bytes); int classNameLength = byteBuf.readInt(); ByteBuf classNameBuf = byteBuf.readBytes(classNameLength); byte[] clazzNameBytes = new byte[classNameBuf.readableBytes()]; classNameBuf.readBytes(clazzNameBytes); //实例化 String className = new String(clazzNameBytes, StandardCharsets.UTF_8); Class clazz = (Class)Class.forName(className); int objectLength = byteBuf.readInt(); ByteBuf objBuf = byteBuf.readBytes(objectLength); byte[] objectBytes = new byte[objectLength]; objBuf.readBytes(objectBytes); return Utils.byte2Object(objectBytes, clazz); } } ================================================ FILE: core/src/test/java/org/apache/rocketmq/streams/core/runtime/WindowStateTests.java ================================================ /* * 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.core.runtime; import org.apache.rocketmq.streams.core.window.WindowState; import org.apache.rocketmq.streams.core.Num; import org.apache.rocketmq.streams.core.User; public class WindowStateTests { public static void main(String[] args) throws Throwable { WindowState state = new WindowState<>(); Num num = new Num(); num.setNumber(10); User user = new User(); user.setName("zeni"); state.setKey(num); state.setValue(user); byte[] bytes = WindowState.windowState2Byte(state); WindowState state1 = WindowState.byte2WindowState(bytes); System.out.println(state1); } } ================================================ FILE: core/src/test/java/org/apache/rocketmq/streams/core/state/RocksDBStoreTest.java ================================================ /* * 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.core.state; import org.apache.rocketmq.streams.core.util.ColumnFamilyUtil; import org.apache.rocketmq.streams.core.window.WindowKey; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.util.Utils; import java.nio.charset.StandardCharsets; import java.util.List; public class RocksDBStoreTest { public static void main(String[] args) throws Throwable { RocksDBStore rocksDBStore = new RocksDBStore("test"); // String key = "time@1668249210000@1668249195000"; // String key2 = "ewwwwe@1668249600481@1"; WindowKey key1 = new WindowKey("test1", "keyString1", 10l, 1l); WindowKey key2 = new WindowKey("test1", "keyString2", 20l, 1l); Object value1 = "3"; Object value2 = "2"; byte[] keyBytes = key2Byte(key1); byte[] valueBytes = Utils.object2Byte(value1); byte[] keyBytes2 = key2Byte(key2); byte[] valueBytes2 = Utils.object2Byte(value2); rocksDBStore.put(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes2), keyBytes2, valueBytes2); rocksDBStore.put(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes), keyBytes, valueBytes); byte[] bytes = rocksDBStore.get(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes), keyBytes); Object result = Utils.byte2Object(bytes, Object.class); System.out.println(result); byte[] bytes2 = rocksDBStore.get(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes2), keyBytes2); Object result2 = Utils.byte2Object(bytes2, Object.class); System.out.println(result2); WindowKey searchKey = new WindowKey("test1", "keyString1", 13l, 1l); String operatorName = searchKey.getOperatorName(); List> pairs = rocksDBStore.searchStateLessThanWatermark(operatorName, 11l, RocksDBStoreTest::byte2WindowKey); System.out.println(pairs.size()); } private static WindowKey byte2WindowKey(byte[] source) { String str = new String(source, StandardCharsets.UTF_8); String[] split = Utils.split(str); return new WindowKey(split[0], split[1], Long.parseLong(split[2]), Long.parseLong(split[3])); } private static byte[] key2Byte(WindowKey windowKey) { if (windowKey == null) { return new byte[0]; } return windowKey.toString().getBytes(StandardCharsets.UTF_8); } } ================================================ FILE: core/src/test/java/org/apache/rocketmq/streams/core/util/UtilsTests.java ================================================ /* * 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.core.util; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import java.nio.charset.StandardCharsets; public class UtilsTests { private static final ObjectMapper objectMapper = new ObjectMapper(); public static void main(String[] args) throws JsonProcessingException { byte[] bytes = "tests".getBytes(StandardCharsets.UTF_8); String tests = "tests"; byte[] bytes1 = objectMapper.writeValueAsBytes(new User()); String s = Utils.toHexString(bytes1); System.out.println(s); String s1 = new String(bytes1, StandardCharsets.UTF_8); System.out.println(s1); } static class User { private String name; public String getName() { return name; } public void setName(String name) { this.name = name; } } } ================================================ FILE: example/pom.xml ================================================ rocketmq-streams-all org.apache.rocketmq 1.1.2-SNAPSHOT 4.0.0 rocketmq-streams-example 8 8 org.apache.rocketmq rocketmq-streams ${version} ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/WordCount.java ================================================ /* * 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.examples; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; import java.util.Properties; public class WordCount { public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("wordCount"); builder.source("sourceTopic", total -> { String value = new String(total, StandardCharsets.UTF_8); return new Pair<>(null, value); }) .flatMap((ValueMapperAction>) value -> { String[] splits = value.toLowerCase().split(","); return Arrays.asList(splits); }) .keyBy(value -> value) .count() .toRStream() .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.put(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/WordCountFromFirstOffset.java ================================================ /* * 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.examples; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.metadata.StreamConfig; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; import java.util.Properties; import static org.apache.rocketmq.common.consumer.ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET; public class WordCountFromFirstOffset { public static void main(String[] args) { StreamBuilder builder = new StreamBuilder("wordCount"); builder.source("sourceTopic", total -> { String value = new String(total, StandardCharsets.UTF_8); return new Pair<>(null, value); }) .flatMap((ValueMapperAction>) value -> { String[] splits = value.toLowerCase().split("\\W+"); return Arrays.asList(splits); }) .keyBy(value -> value) .count() .toRStream() .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.put(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); properties.put(StreamConfig.ROCKETMQ_STREAMS_CONSUMER_FORM_WHERE, CONSUME_FROM_FIRST_OFFSET); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/joinWindow/JoinWindow.java ================================================ /* * 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.examples.joinWindow; import com.alibaba.fastjson.JSON; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.function.ValueJoinAction; import org.apache.rocketmq.streams.core.rstream.RStream; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.window.Time; import org.apache.rocketmq.streams.core.window.WindowBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.examples.pojo.Num; import org.apache.rocketmq.streams.examples.pojo.Union; import org.apache.rocketmq.streams.examples.pojo.User; import java.util.Properties; /** * 1、启动RocketMQ * 2、创建topic * 3、启动本例子运行 * 4、向topic中写入数据 * 5、观察输出结果 */ public class JoinWindow { public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("joinWindow"); RStream user = builder.source("user", total -> { User user1 = JSON.parseObject(total, User.class); return new Pair<>(null, user1); }); RStream num = builder.source("num", source -> { Num user12 = JSON.parseObject(source, Num.class); return new Pair<>(null, user12); }); ValueJoinAction action = new ValueJoinAction() { @Override public Union apply(User value1, Num value2) { if (value1 != null && value2 != null) { System.out.println("name in user: " + value1.getName()); System.out.println("name in num: " + value2.getName()); return new Union(value1.getName(), value1.getAge(), value2.getNum()); } if (value2 != null) { System.out.println("name in num: " + value2.getName()); return new Union(value2.getName(), 0, value2.getNum()); } if (value1 != null) { System.out.println("name in num: " + value1.getName()); return new Union(value1.getName(), value1.getAge(), 0); } throw new IllegalStateException(); } }; user.join(num) .where(User::getName) .equalTo(Num::getName) .window(WindowBuilder.tumblingWindow(Time.seconds(10))) .apply(action) .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.put(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/joinWindow/JoinWithoutWindow.java ================================================ /* * 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.examples.joinWindow; import com.alibaba.fastjson.JSON; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.function.ValueJoinAction; import org.apache.rocketmq.streams.core.rstream.RStream; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.examples.pojo.Num; import org.apache.rocketmq.streams.examples.pojo.Union; import org.apache.rocketmq.streams.examples.pojo.User; import java.util.Properties; public class JoinWithoutWindow { public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("JoinWithoutWindow"); RStream user = builder.source("user", total -> { User user1 = JSON.parseObject(total, User.class); return new Pair<>(null, user1); }); RStream num = builder.source("num", source -> { Num user12 = JSON.parseObject(source, Num.class); return new Pair<>(null, user12); }); ValueJoinAction action = new ValueJoinAction() { @Override public Union apply(User value1, Num value2) { if (value1 != null && value2 != null) { System.out.println("name in user: " + value1.getName()); System.out.println("name in num: " + value2.getName()); return new Union(value1.getName(), value1.getAge(), value2.getNum()); } if (value2 != null) { System.out.println("name in num: " + value2.getName()); return new Union(value2.getName(), 0, value2.getNum()); } if (value1 != null) { System.out.println("name in num: " + value1.getName()); return new Union(value1.getName(), value1.getAge(), 0); } throw new IllegalStateException(); } }; user.join(num) .where(User::getName) .equalTo(Num::getName) .apply(action) .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.put(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Demo.java ================================================ package org.apache.rocketmq.streams.examples.pojo; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import com.alibaba.fastjson.JSON; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.serialization.KeyValueDeserializer; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import java.util.Properties; /** * 1、启动RocketMQ * 2、创建topic * 3、启动本例子运行 * 4、向topic中写入数据 * 5、观察输出结果 */ public class Demo { public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("demo"); builder.source("user", new KeyValueDeserializer() { @Override public Pair deserialize(byte[] total) throws Throwable { //对象需要有默认构造器 User user = JSON.parseObject(total, User.class); return new Pair<>(null, user); } }) .keyBy(User::getAge) .count(User::getName) .toRStream() .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.put(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Grade.java ================================================ /* * 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.examples.pojo; public class Grade { private int grade; private int score; private long timestamp; public Grade(int grade, int score) { this.grade = grade; this.score = score; } public Grade() { } public int getGrade() { return grade; } public void setGrade(int grade) { this.grade = grade; } public int getScore() { return score; } public void setScore(int score) { this.score = score; } public long getTimestamp() { return timestamp; } public void setTimestamp(long timestamp) { this.timestamp = timestamp; } @Override public String toString() { return "Grade{" + "grade=" + grade + ", score=" + score + ", timestamp=" + timestamp + '}'; } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Num.java ================================================ /* * 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.examples.pojo; public class Num { private String name; private Integer num; public Num(){ } public Num(String name, Integer num) { this.name = name; this.num = num; } public String getName() { return name; } public void setName(String name) { this.name = name; } public Integer getNum() { return num; } public void setNum(Integer num) { this.num = num; } @Override public String toString() { return "Num{" + "name='" + name + '\'' + ", num=" + num + '}'; } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Order.java ================================================ package org.apache.rocketmq.streams.examples.pojo; public class Order { private String type; // drink, food, private Integer price; // order price private String customer; // customer name public Order() { } public Order(String type, Integer price, String customer) { this.type = type; this.price = price; this.customer = customer; } public String getType() { return type; } public String getCustomer() { return customer; } public Integer getPrice() { return price; } public void setType(String type) { this.type = type; } public void setCustomer(String customer) { this.customer = customer; } public void setPrice(Integer price) { this.price = price; } @Override public String toString() { return "Order{" + "type=" + type + ", price=" + price + ", customer=" + customer + "}"; } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Union.java ================================================ /* * 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.examples.pojo; public class Union { private String name; private Integer age; private Integer num; public Union(String name, Integer age, Integer num) { this.name = name; this.age = age; this.num = num; } public Union() { } public String getName() { return name; } public void setName(String name) { this.name = name; } public Integer getAge() { return age; } public void setAge(Integer age) { this.age = age; } public Integer getNum() { return num; } public void setNum(Integer num) { this.num = num; } @Override public String toString() { return "Union{" + "name='" + name + '\'' + ", age=" + age + ", num=" + num + '}'; } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/User.java ================================================ /* * 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.examples.pojo; public class User { private String name; private Integer age; private long timestamp; //序列化/反序列化使用 public User() { } public User(String name, Integer age) { this.name = name; this.age = age; } public User(String name, Integer age, long timestamp) { this.name = name; this.age = age; this.timestamp = timestamp; } public String getName() { return name; } public void setName(String name) { this.name = name; } public Integer getAge() { return age; } public void setAge(Integer age) { this.age = age; } public long getTimestamp() { return timestamp; } public void setTimestamp(long timestamp) { this.timestamp = timestamp; } @Override public String toString() { return "User{" + "name='" + name + '\'' + ", age=" + age + ", timeStamp=" + timestamp + '}'; } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/sink/WordCountSink.java ================================================ package org.apache.rocketmq.streams.examples.sink; /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.serialization.KeyValueSerializer; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; import java.util.Properties; /** * 1、启动RocketMQ * 2、创建topic sh bin/mqadmin updateTopic -c DefaultCluster -t sourceTopic -r 8 -w 8 -n 127.0.0.1:9876 * 3、启动本例子运行 * 4、向topic中写入数据 * 5、观察输出结果 */ public class WordCountSink { public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("wordCount"); builder.source("sourceTopic", total -> { String value = new String(total, StandardCharsets.UTF_8); return new Pair<>(null, value); }) .flatMap((ValueMapperAction>) value -> { String[] splits = value.toLowerCase().split("\\W+"); return Arrays.asList(splits); }) .keyBy(value -> value) .count() .sink("wordCountSink", new KeyValueSerializer() { final ObjectMapper objectMapper = new ObjectMapper(); @Override public byte[] serialize(String o, Integer data) throws Throwable { ObjectNode objectNode = objectMapper.createObjectNode(); objectNode.put(o, data); String result = objectNode.toPrettyString(); return objectMapper.writeValueAsBytes(result); } }); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.put(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/SessionWindowCount.java ================================================ /* * 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.examples.window; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.window.Time; import org.apache.rocketmq.streams.core.window.WindowBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import java.nio.charset.StandardCharsets; import java.text.SimpleDateFormat; import java.util.Date; import java.util.Properties; /** * 1、启动RocketMQ * 2、创建topic * 3、启动本例子运行 * 4、向topic中写入数据 * 5、观察输出结果 */ public class SessionWindowCount { private static final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("sessionWindowCount"); builder.source("windowCount", source -> { String value = new String(source, StandardCharsets.UTF_8); int result = Integer.parseInt(value); System.out.println("time=" + format.format(new Date(System.currentTimeMillis())) + ", value=" + value); return new Pair<>(null, result); }) .filter(value -> value > 0) .keyBy(value -> "key") .window(WindowBuilder.sessionWindow(Time.seconds(5))) .count() .toRStream() .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/SlideWindowCount.java ================================================ /* * 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.examples.window; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.window.Time; import org.apache.rocketmq.streams.core.window.WindowBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import java.nio.charset.StandardCharsets; import java.util.Properties; /** * 1、启动RocketMQ * 2、创建topic * 3、启动本例子运行 * 4、向topic中写入数据 * 5、观察输出结果 */ public class SlideWindowCount { public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("slideWindowCount"); builder.source("windowCount", source -> { String value = new String(source, StandardCharsets.UTF_8); int result = Integer.parseInt(value); return new Pair<>(null, result); }) .filter(value -> value > 0) .keyBy(value -> "key") .window(WindowBuilder.slidingWindow(Time.seconds(5), Time.seconds(2))) .count() .toRStream() .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowAvg.java ================================================ /* * 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.examples.window; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.window.Time; import org.apache.rocketmq.streams.core.window.WindowBuilder; import java.nio.charset.StandardCharsets; import java.time.LocalTime; import java.util.Properties; import java.util.concurrent.CountDownLatch; /** * 1、启动RocketMQ * 2、创建topic * 3、启动本例子运行 * 4、向topic中写入数据 * 5、观察输出结果 */ public class WindowAvg { public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("WindowAvg"); builder.source("avgSource", source -> { String value = new String(source, StandardCharsets.UTF_8); Integer num = Integer.parseInt(value); return new Pair<>(null, num); }).foreach(value -> System.out.println(String.format("time:%s, input:%d", LocalTime.now(), value))) .filter(value -> value > 0) .keyBy(value -> "key") .window(WindowBuilder.tumblingWindow(Time.seconds(5))) .avg() .toRStream() .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("WindowAvg-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowCount.java ================================================ /* * 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.examples.window; import com.alibaba.fastjson.JSON; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.metadata.StreamConfig; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.window.Time; import org.apache.rocketmq.streams.core.window.TimeType; import org.apache.rocketmq.streams.core.window.WindowBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.examples.pojo.User; import java.util.Properties; /** * 1、启动RocketMQ * 2、创建topic * 3、启动本例子运行 * 4、向topic中写入数据 * 5、观察输出结果 */ public class WindowCount { public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("windowCountUser"); builder.source("user", source -> { User user = JSON.parseObject(source, User.class); return new Pair<>(null, user); }) .selectTimestamp(User::getTimestamp) .keyBy(User::getAge) .window(WindowBuilder.tumblingWindow(Time.seconds(5))) .count() .toRStream() .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); properties.put(StreamConfig.TIME_TYPE, TimeType.EVENT_TIME); properties.put(StreamConfig.ALLOW_LATENESS_MILLISECOND, 2000); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowMin.java ================================================ /* * 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.examples.window; import com.alibaba.fastjson.JSON; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.metadata.StreamConfig; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.window.Time; import org.apache.rocketmq.streams.core.window.WindowBuilder; import org.apache.rocketmq.streams.examples.pojo.Grade; import java.text.SimpleDateFormat; import java.util.Date; import java.util.Properties; /** * 1、启动RocketMQ * 2、创建topic * 3、启动本例子运行 * 4、向topic中写入数据 * 5、观察输出结果 */ public class WindowMin { private static final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("windowMinGrade"); builder.source("grade", source -> { Grade grade = JSON.parseObject(source, Grade.class); System.out.println("time=" + format.format(new Date(System.currentTimeMillis())) + "," + grade); return new Pair<>(null, grade); }) .keyBy(Grade::getGrade) .window(WindowBuilder.tumblingWindow(Time.seconds(5))) .min(Grade::getScore) //.max(Grade::getScore) .toRStream() .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); properties.put(StreamConfig.ALLOW_LATENESS_MILLISECOND, 2000); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowOrderCount.java ================================================ package org.apache.rocketmq.streams.examples.window; import com.alibaba.fastjson.JSON; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.function.SelectAction; import org.apache.rocketmq.streams.core.metadata.StreamConfig; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.window.Time; import org.apache.rocketmq.streams.core.window.WindowBuilder; import org.apache.rocketmq.streams.examples.pojo.Order; import java.util.Properties; public class WindowOrderCount { public static void main(String[] args) throws Throwable { StreamBuilder builder = getOrder2(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); properties.put(StreamConfig.ALLOW_LATENESS_MILLISECOND, 2000); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("ordercount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } private static StreamBuilder getOrder1() throws Throwable { /** * Get the count of drink/food orders in last 30 seconds every 10 seconds **/ StreamBuilder builder = new StreamBuilder("windowOrderCount"); builder.source("order", source -> { Order order = JSON.parseObject(source, Order.class); System.out.println(order.toString()); return new Pair<>(null, order); }) .keyBy(Order::getType) .window(WindowBuilder.slidingWindow(Time.seconds(30), Time.seconds(10))) .count() .toRStream() .print(); return builder; } private static StreamBuilder getOrder2() throws Throwable { /** * Get how much the customers pay for drink/food every 100 seconds **/ StreamBuilder builder = new StreamBuilder("windowOrderCount"); builder.source("order", source -> { Order order = JSON.parseObject(source, Order.class); System.out.println(order.toString()); return new Pair<>(null, order); }) .keyBy(new SelectAction() { @Override public String select(Order order) { return order.getCustomer() + "@" + order.getType(); } }) .window(WindowBuilder.tumblingWindow(Time.seconds(100))) .sum(Order::getPrice) .toRStream() .print(); return builder; } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowSum.java ================================================ /* * 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.examples.window; import com.alibaba.fastjson.JSON; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.metadata.StreamConfig; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import org.apache.rocketmq.streams.core.window.Time; import org.apache.rocketmq.streams.core.window.WindowBuilder; import org.apache.rocketmq.streams.examples.pojo.Grade; import java.text.SimpleDateFormat; import java.util.Date; import java.util.Properties; /** * 1、启动RocketMQ * 2、创建topic * 3、启动本例子运行 * 4、向topic中写入数据 * 5、观察输出结果 */ public class WindowSum { private static final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("windowSumGrade"); builder.source("grade", source -> { Grade grade = JSON.parseObject(source, Grade.class); System.out.println("time=" + format.format(new Date(System.currentTimeMillis())) + "," + grade); return new Pair<>(null, grade); }) .keyBy(Grade::getGrade) .window(WindowBuilder.tumblingWindow(Time.seconds(5))) .sum(Grade::getScore) .toRStream() .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); properties.put(StreamConfig.ALLOW_LATENESS_MILLISECOND, 2000); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowWordCount.java ================================================ /* * 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.examples.window; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.streams.core.RocketMQStream; import org.apache.rocketmq.streams.core.function.ValueMapperAction; import org.apache.rocketmq.streams.core.metadata.StreamConfig; import org.apache.rocketmq.streams.core.rstream.StreamBuilder; import org.apache.rocketmq.streams.core.window.Time; import org.apache.rocketmq.streams.core.window.TimeType; import org.apache.rocketmq.streams.core.window.WindowBuilder; import org.apache.rocketmq.streams.core.topology.TopologyBuilder; import org.apache.rocketmq.streams.core.util.Pair; import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; import java.util.Properties; /** * 1、启动RocketMQ * 2、创建topic * 3、启动本例子运行 * 4、向topic中写入数据 * 5、观察输出结果 */ public class WindowWordCount { public static void main(String[] args) throws Throwable { StreamBuilder builder = new StreamBuilder("windowWordCount"); builder.source("sourceTopic", source -> { String value = new String(source, StandardCharsets.UTF_8); return new Pair<>(null, value); }) .flatMap((ValueMapperAction>) value -> { String[] splits = value.toLowerCase().split("\\W+"); return Arrays.asList(splits); }) .keyBy(value -> value) .window(WindowBuilder.tumblingWindow(Time.seconds(5))) .count() .toRStream() .print(); TopologyBuilder topologyBuilder = builder.build(); Properties properties = new Properties(); properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876"); properties.put(StreamConfig.TIME_TYPE, TimeType.PROCESS_TIME); RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties); Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") { @Override public void run() { rocketMQStream.stop(); } }); rocketMQStream.start(); } } ================================================ FILE: pom.xml ================================================ 4.0.0 org.apache apache 18 org.apache.rocketmq rocketmq-streams-all 1.1.2-SNAPSHOT Apache RocketMQ Streams ${project.version} pom https://rocketmq.apache.org/ git@github.com:apache/rocketmq-streams.git scm:git:git@github.com:apache/rocketmq-streams.git scm:git:git@github.com:apache/rocketmq-streams.git HEAD Development List dev-subscribe@rocketmq.apache.org dev-unsubscribe@rocketmq.apache.org dev@rocketmq.apache.org User List users-subscribe@rocketmq.apache.org users-unsubscribe@rocketmq.apache.org users@rocketmq.apache.org Commits List commits-subscribe@rocketmq.apache.org commits-unsubscribe@rocketmq.apache.org commits@rocketmq.apache.org Apache RocketMQ Apache RocketMQ of ASF https://rocketmq.apache.org/ Apache Software Foundation http://www.apache.org Apache License, Version 2.0 http://www.apache.org/licenses/LICENSE-2.0 repo core example 5.0.0 7.6.0 8 8 org.apache.rocketmq rocketmq-tools ${rocketmq.version} org.apache.rocketmq rocketmq-common ${rocketmq.version} org.apache.rocketmq rocketmq-client ${rocketmq.version} org.apache.rocketmq rocketmq-logging ${rocketmq.version} org.rocksdb rocksdbjni ${rocksdbjni.version} org.apache.maven.plugins maven-surefire-plugin 2.18.1 true org.apache.rat apache-rat-plugin 0.12 .gitignore .travis.yml .asf.yaml build_without_test.sh NOTICE LICENSE README.md .github/** */target/** */*.iml **/*.txt **/*.cs **/*.xml **/*.sh **/*.out **/*.properties org.apache.maven.plugins maven-compiler-plugin 3.5.1 ${maven.compiler.source} ${maven.compiler.target} true true org.codehaus.mojo versions-maven-plugin 2.2 org.codehaus.mojo cobertura-maven-plugin 2.7 html xml org.jacoco jacoco-maven-plugin 0.8.7 prepare-agent report test report org.apache.maven.plugins maven-source-plugin 3.0.1 attach-sources jar