Full Code of apache/rocketmq-streams for AI

develop 22976de3ef0f cached
132 files
452.2 KB
102.6k tokens
803 symbols
1 requests
Download .txt
Showing preview only (503K chars total). Download the full file or copy to clipboard to get everything.
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
    <dependency>
        <groupId>org.apache.rocketmq</groupId>
        <artifactId>rocketmq-streams</artifactId>
        <version>{current.version}</version>
    </dependency>
```

**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<String, List<String>>) 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
================================================
<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    <parent>
        <artifactId>rocketmq-streams-all</artifactId>
        <groupId>org.apache.rocketmq</groupId>
        <version>1.1.2-SNAPSHOT</version>
    </parent>
    <modelVersion>4.0.0</modelVersion>

    <artifactId>rocketmq-streams</artifactId>
    <packaging>jar</packaging>

    <properties>
        <maven.compiler.source>8</maven.compiler.source>
        <maven.compiler.target>8</maven.compiler.target>
    </properties>

    <dependencies>
        <dependency>
            <groupId>org.apache.rocketmq</groupId>
            <artifactId>rocketmq-common</artifactId>
            <version>${rocketmq.version}</version>
        </dependency>
        <dependency>
            <groupId>org.apache.rocketmq</groupId>
            <artifactId>rocketmq-client</artifactId>
            <version>${rocketmq.version}</version>
        </dependency>

        <dependency>
            <groupId>org.apache.rocketmq</groupId>
            <artifactId>rocketmq-tools</artifactId>
            <exclusions>
                <exclusion>
                    <groupId>com.google.protobuf</groupId>
                    <artifactId>protobuf-java</artifactId>
                </exclusion>
            </exclusions>
            <version>${rocketmq.version}</version>
        </dependency>


        <dependency>
            <groupId>org.rocksdb</groupId>
            <artifactId>rocksdbjni</artifactId>
            <version>7.6.0</version>
        </dependency>
        <dependency>
            <groupId>commons-io</groupId>
            <artifactId>commons-io</artifactId>
            <version>2.11.0</version>
        </dependency>

        <dependency>
            <groupId>com.fasterxml.jackson.core</groupId>
            <artifactId>jackson-databind</artifactId>
            <version>2.13.4.2</version>
        </dependency>

    </dependencies>

</project>

================================================
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<WorkerThread> 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<K, V, OUT> {
    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<T> {
    /**
     *
     * @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<T> {
    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, IN> {
    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<V1, V2, OUT> {

    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<T, O> {
    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<V, R> {
    void addValue(V value);

    void merge(Accumulator<V, R> other);

    /**
     * invoked after the window fired.
     *
     * @param context the attached properties after window fired.
     * @return the value.
     */
    R result(Properties context);

    Accumulator<V, R> 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<V> implements Accumulator<V, Double> {
    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<V, Double> other) {
        if (other instanceof AvgAccumulator) {
            AvgAccumulator<V> 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<V, Double> 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<V> implements Accumulator<V, Integer> {
    private Integer count = 0;

    @Override
    public void addValue(V value) {
        count += 1;
    }

    @Override
    public void merge(Accumulator<V, Integer> 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<V, Integer> clone() {
        CountAccumulator<V> 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<V> implements Accumulator<V, Double> {
     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<V, Double> other) {
        min = Math.min(min.doubleValue(), other.result(null));
    }

    @Override
    public Double result(Properties context) {
        return min.doubleValue();
    }

    @Override
    public Accumulator<V, Double> 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<K, V, R, OV> implements Supplier<Processor<V>> {
    private final String currentName;
    private final String parentName;
    private SelectAction<R, V> selectAction;
    private Accumulator<R, OV> accumulator;

    public AccumulatorSupplier(String currentName, String parentName, SelectAction<R, V> selectAction, Accumulator<R, OV> accumulator) {
        this.currentName = currentName;
        this.parentName = parentName;
        this.selectAction = selectAction;
        this.accumulator = accumulator;
    }

    @Override
    public Processor<V> get() {
        return new AccumulatorProcessor(currentName, parentName, selectAction, accumulator);
    }

    private class AccumulatorProcessor extends AbstractProcessor<V> {
        private final String currentName;
        private final String parentName;
        private StateStore stateStore;
        private MessageQueue stateTopicMessageQueue;
        private SelectAction<R, V> selectAction;
        private Accumulator<R, OV> accumulator;

        public AccumulatorProcessor(String currentName, String parentName, SelectAction<R, V> selectAction, Accumulator<R, OV> accumulator) {
            this.currentName = currentName;
            this.parentName = parentName;
            this.selectAction = selectAction;
            this.accumulator = accumulator;
        }

        @Override
        public void preProcess(StreamContext<V> 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<R, OV> 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<K, OV> temp = new Data<>(key, result, this.context.getDataTime(), this.context.getHeader());
            Data<K, V> 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<T> implements Supplier<Processor<T>> {
    private final Properties properties = new Properties();

    public AddTagSupplier(Supplier<Object> value) {

        properties.put(Constant.STREAM_TAG, value.get());
    }

    public AddTagSupplier() {
    }

    @Override
    public Processor<T> get() {
        return new AddTagProcessor(properties);
    }

    class AddTagProcessor extends AbstractProcessor<T> {
        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<Object, T> 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<K, V, OV> implements Supplier<Processor<V>> {
    private final String currentName;
    private final String parentName;
    private Supplier<OV> initAction;
    private AggregateAction<K, V, OV> aggregateAction;

    public AggregateSupplier(String currentName, String parentName, Supplier<OV> initAction,
                             AggregateAction<K, V, OV> aggregateAction) {
        this.currentName = currentName;
        this.parentName = parentName;
        this.initAction = initAction;
        this.aggregateAction = aggregateAction;
    }

    @Override
    public Processor<V> get() {
        return new AggregateProcessor(currentName, parentName, initAction, aggregateAction);
    }

    private class AggregateProcessor extends AbstractProcessor<V> {
        private final String currentName;
        private final String parentName;
        private final Supplier<OV> initAction;
        private final AggregateAction<K, V, OV> aggregateAction;
        private StateStore stateStore;
        private MessageQueue stateTopicMessageQueue;

        public AggregateProcessor(String currentName, String parentName, Supplier<OV> initAction,
                                  AggregateAction<K, V, OV> aggregateAction) {
            this.currentName = currentName;
            this.parentName = parentName;
            this.initAction = initAction;
            this.aggregateAction = aggregateAction;
        }

        @Override
        public void preProcess(StreamContext<V> 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<K, OV> temp = new Data<>(key, result, this.context.getDataTime(), this.context.getHeader());
            Data<K, V> 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<T> implements Supplier<Processor<T>> {
    private FilterAction<T> filterAction;

    public FilterSupplier(FilterAction<T> filterAction) {
        this.filterAction = filterAction;
    }

    @Override
    public Processor<T> get() {
        return new FilterProcessor(filterAction);
    }

    private class FilterProcessor extends AbstractProcessor<T> {
        private final FilterAction<T> filterAction;

        public FilterProcessor(FilterAction<T> filterAction) {
            this.filterAction = filterAction;
        }

        @Override
        public  void process(T data) throws Throwable {
            boolean pass = filterAction.apply(data);
            if (pass) {
                Data<Object, T> 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<T> implements Supplier<Processor<T>> {
    private ForeachAction<T> foreachAction;

    public ForeachSupplier(ForeachAction<T> foreachAction) {
        this.foreachAction = foreachAction;
    }

    @Override
    public Processor<T> get() {
        return new ForeachProcessor(this.foreachAction);
    }

    class ForeachProcessor extends AbstractProcessor<T> {
        private ForeachAction<T> foreachAction;

        public ForeachProcessor(ForeachAction<T> foreachAction) {
            this.foreachAction = foreachAction;
        }

        @Override
        public void process(T data) throws Throwable {
            this.foreachAction.apply(data);
            Data<Object, T> 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<K, V1, V2, OUT> implements Supplier<Processor<? super OUT>> {
    private static final Logger logger = LoggerFactory.getLogger(JoinAggregateSupplier.class.getName());

    private String name;
    private JoinType joinType;
    private final ValueJoinAction<V1, V2, OUT> joinAction;

    public JoinAggregateSupplier(String name, JoinType joinType, ValueJoinAction<V1, V2, OUT> joinAction) {
        this.name = name;
        this.joinType = joinType;
        this.joinAction = joinAction;
    }

    @Override
    public Processor<Object> get() {
        return new JoinStreamAggregateProcessor(name, joinType, joinAction);
    }

    private class JoinStreamAggregateProcessor extends AbstractProcessor<Object> {
        private String name;
        private JoinType joinType;
        private final ValueJoinAction<V1, V2, OUT> joinAction;
        private MessageQueue stateTopicMessageQueue;
        private StateStore stateStore;


        public JoinStreamAggregateProcessor(String name, JoinType joinType, ValueJoinAction<V1, V2, OUT> joinAction) {
            this.name = name;
            this.joinType = joinType;
            this.joinAction = joinAction;
        }

        @Override
        public void preProcess(StreamContext<Object> 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<K, OUT> result = new Data<>(this.context.getKey(), out, this.context.getDataTime(), this.context.getHeader());
            Data<K, Object> 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<K, V1, V2, OUT> implements Supplier<Processor<? super OUT>> {
    private static final Logger logger = LoggerFactory.getLogger(JoinWindowAggregateSupplier.class.getName());

    private String name;
    private WindowInfo windowInfo;
    private final ValueJoinAction<V1, V2, OUT> joinAction;
    private JoinType joinType;

    public JoinWindowAggregateSupplier(String name, WindowInfo windowInfo, ValueJoinAction<V1, V2, OUT> joinAction) {
        this.name = name;
        this.windowInfo = windowInfo;
        this.joinType = windowInfo.getJoinStream().getJoinType();
        this.joinAction = joinAction;
    }

    @Override
    public Processor<Object> get() {
        return new JoinStreamWindowAggregateProcessor(name, windowInfo, joinType, joinAction);
    }


    @SuppressWarnings("unchecked")
    private class JoinStreamWindowAggregateProcessor extends AbstractWindowProcessor<Object> {
        private String name;
        private final WindowInfo windowInfo;
        private final JoinType joinType;
        private ValueJoinAction<V1, V2, OUT> joinAction;
        private MessageQueue stateTopicMessageQueue;
        private WindowStore<K, V1> leftWindowStore;
        private WindowStore<K, V2> rightWindowStore;

        public JoinStreamWindowAggregateProcessor(String name, WindowInfo windowInfo, JoinType joinType, ValueJoinAction<V1, V2, OUT> joinAction) {
            this.name = Utils.buildKey(name, JoinStreamWindowAggregateProcessor.class.getSimpleName());
            this.windowInfo = windowInfo;
            this.joinType = joinType;
            this.joinAction = joinAction;
        }

        @Override
        public void preProcess(StreamContext<Object> 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<WindowKey> 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<Window> 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<K, V1> 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<K, V2> 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<KEY, T> implements Supplier<Processor<T>> {
    private final SelectAction<KEY, T> selectAction;

    public KeySelectSupplier(SelectAction<KEY, T> selectAction) {
        this.selectAction = selectAction;
    }

    @Override
    public Processor<T> get() {
        return new MapperProcessor(selectAction);
    }

    private class MapperProcessor extends AbstractProcessor<T> {
        private final SelectAction<KEY, T> selectAction;


        public MapperProcessor(SelectAction<KEY, T> selectAction) {
            this.selectAction = selectAction;
        }

        @Override
        public void process(T data) throws Throwable {
            KEY newKey = selectAction.select(data);
            Data<KEY, T> 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<T, VR> implements Supplier<Processor<T>> {
    private final ValueMapperAction<T, ? extends Iterable<? extends VR>> valueMapperAction;
    private static final Logger logger = LoggerFactory.getLogger(MultiValueChangeSupplier.class.getName());

    public MultiValueChangeSupplier(ValueMapperAction<T, ? extends Iterable<? extends VR>> valueMapperAction) {
        this.valueMapperAction = valueMapperAction;
    }

    @Override
    public Processor<T> get() {
        return new MultiValueMapperProcessor<>(this.valueMapperAction);
    }

    static class MultiValueMapperProcessor<T, VR> extends AbstractProcessor<T> {
        private final ValueMapperAction<T, ? extends Iterable<? extends VR>> valueMapperAction;

        public MultiValueMapperProcessor(ValueMapperAction<T, ? extends Iterable<? extends VR>> valueMapperAction) {
            this.valueMapperAction = valueMapperAction;
        }

        @Override
        public void process(T data) throws Throwable {
            Iterable<? extends VR> 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<Object, VR> before = new Data<>(this.context.getKey(), item, this.context.getDataTime(), this.context.getHeader());
                Data<Object, T> 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<T> implements Supplier<Processor<T>> {


    @Override
    public Processor<T> get() {
        return new PrintProcessor<>();
    }

    static class PrintProcessor<T> extends AbstractProcessor<T> {


        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<Object, T> 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<K, T> implements Supplier<Processor<T>> {
    private static final Logger logger = LoggerFactory.getLogger(SinkSupplier.class);

    private final String topicName;
    private final KeyValueSerializer<K, T> serializer;

    public SinkSupplier(String topicName, KeyValueSerializer<K, T> serializer) {
        this.topicName = topicName;
        this.serializer = serializer;
    }

    @Override
    public Processor<T> get() {
        return new SinkProcessor(this.topicName, this.serializer);
    }

    private class SinkProcessor extends AbstractProcessor<T> {
        private final String topicName;
        private DefaultMQProducer producer;
        private final KeyValueSerializer<K, T> serializer;
        private K key;

        public SinkProcessor(String topicName, KeyValueSerializer<K, T> serializer) {
            this.topicName = topicName;
            this.serializer = serializer;
        }

        @Override
        public void preProcess(StreamContext<T> 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<K, V> implements Supplier<Processor<V>> {
    private String topicName;
    private KeyValueDeserializer<K, V> deserializer;

    public SourceSupplier(String topicName, KeyValueDeserializer<K, V> deserializer) {
        this.topicName = topicName;
        this.deserializer = deserializer;
    }

    @Override
    public Processor<V> get() {
        return new SourceProcessorImpl(deserializer);
    }

    public interface SourceProcessor<K, V> extends Processor<V> {
        Pair<K, V> deserialize(String keyClass, String valueClass, byte[] data) throws DeserializeThrowable;

        long getTimestamp(MessageExt originData, TimeType timeType);
    }

    private class SourceProcessorImpl extends AbstractProcessor<V> implements SourceProcessor<K, V> {
        private KeyValueDeserializer<K, V> deserializer;

        public SourceProcessorImpl(KeyValueDeserializer<K, V> deserializer) {
            this.deserializer = deserializer;
        }

        @Override
        public Pair<K, V> 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<K, V> implements AggregateAction<K, V, Number> {
    private final SelectAction<? extends Number, V> selectAction;

    public SumAggregate(SelectAction<? extends Number, V> 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<T> implements Supplier<Processor<T>> {
    private final ValueMapperAction<T, Long> valueMapperAction;

    public TimestampSelectorSupplier(ValueMapperAction<T, Long> valueMapperAction) {
        this.valueMapperAction = valueMapperAction;
    }

    @Override
    public Processor<T> get() {
        return new TimestampSelector<>(valueMapperAction);
    }


    static class TimestampSelector<T> extends AbstractProcessor<T> {
        private final ValueMapperAction<T, Long> valueMapperAction;

        public TimestampSelector(ValueMapperAction<T, Long> valueMapperAction) {
            this.valueMapperAction = valueMapperAction;
        }

        @Override
        public void process(T data) throws Throwable {
            Long timestamp = this.valueMapperAction.convert(data);

            StreamContext<T> streamContext = this.context;

            //override the timestamp of data
            Data<Object, T> 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<T, O> implements Supplier<Processor<T>> {
    private final ValueMapperAction<T, O> valueMapperAction;
    private static final Logger logger = LoggerFactory.getLogger(ValueChangeSupplier.class.getName());


    public ValueChangeSupplier(ValueMapperAction<T, O> valueMapperAction) {
        this.valueMapperAction = valueMapperAction;
    }

    @Override
    public Processor<T> get() {
        return new ValueMapperProcessor<>(this.valueMapperAction);
    }


    static class ValueMapperProcessor<T, O> extends AbstractProcessor<T> {
        private final ValueMapperAction<T, O> valueMapperAction;


        public ValueMapperProcessor(ValueMapperAction<T, O> 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<Object, O> before = new Data<>(this.context.getKey(), convert, this.context.getDataTime(), this.context.getHeader());
            Data<Object, T> 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<K, V, R, OV> implements Supplier<Processor<V>> {
    private static final Logger logger = LoggerFactory.getLogger(WindowAccumulatorSupplier.class.getName());
    private final String name;
    private WindowInfo windowInfo;
    private SelectAction<R, V> selectAction;
    private Accumulator<R, OV> accumulator;

    public WindowAccumulatorSupplier(String name, WindowInfo windowInfo,
                                     SelectAction<R, V> selectAction, Accumulator<R, OV> accumulator) {
        this.name = name;
        this.windowInfo = windowInfo;
        this.selectAction = selectAction;
        this.accumulator = accumulator;
    }

    @Override
    public Processor<V> 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<V> {
        private final WindowInfo windowInfo;
        private String name;
        private MessageQueue stateTopicMessageQueue;
        private SelectAction<R, V> selectAction;
        private Accumulator<R, OV> accumulator;
        private WindowStore<K, Accumulator<R, OV>> windowStore;

        private final AtomicReference<Throwable> errorReference = new AtomicReference<>(null);

        public WindowAccumulatorProcessor(String name, WindowInfo windowInfo, SelectAction<R, V> selectAction, Accumulator<R, OV> 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<V> 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<Window>
            List<Window> 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<K, Accumulator<R, OV>> oldState = this.windowStore.get(windowKey);

                //f(oldValue, Agg) -> newValue
                Accumulator<R, OV> 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<K, Accumulator<R, OV>> state = new WindowState<>(key, storeAccumulator, time);
                this.windowStore.put(stateTopicMessageQueue, windowKey, state);
                this.idleWindowScaner.putAccumulatorWindowCallback(windowKey, watermark, this.accumulatorWindowFire);
            }

            try {
                List<WindowKey> 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<V> {
        private final String name;
        private final WindowInfo windowInfo;
        private MessageQueue stateTopicMessageQueue;
        private SelectAction<R, V> selectAction;
        private Accumulator<R, OV> accumulator;
        private WindowStore<K, Accumulator<R, OV>> windowStore;

        public SessionWindowAccumulatorProcessor(String name, WindowInfo windowInfo, SelectAction<R, V> selectAction, Accumulator<R, OV> 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<V> 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<Long, Long> newSessionWindowTime = fireIfSessionOut(key, data, time, watermark);

            if (newSessionWindowTime != null) {
                Accumulator<R, OV> temp = accumulator.clone();
                R select = selectAction.select(data);
                temp.addValue(select);

                WindowState<K, Accumulator<R, OV>> 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<Long/*sessionBegin*/, Long/*sessionEnd*/> fireIfSessionOut(K key, V data, long dataTime, long watermark) throws Throwable {
            List<Pair<WindowKey, WindowState<K, Accumulator<R, OV>>>> 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<Pair<WindowKey, WindowState<K, Accumulator<R, OV>>>> iterator = pairs.iterator();
            int count = 0;
            long lastStateSessionEnd = 0;
            long maxFireSessionEnd = Long.MIN_VALUE;

            while (iterator.hasNext()) {
                Pair<WindowKey, WindowState<K, Accumulator<R, OV>>> 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<WindowKey> 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<WindowKey, WindowState<K, Accumulator<R, OV>>> pair = pairs.get(i);

                WindowKey windowKey = pair.getKey();
                WindowState<K, Accumulator<R, OV>> 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<R, OV> 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<K, V, OV> implements Supplier<Processor<V>> {
    private static final Logger logger = LoggerFactory.getLogger(WindowAggregateSupplier.class.getName());
    private String name;
    private WindowInfo windowInfo;
    private Supplier<OV> initAction;
    private AggregateAction<K, V, OV> aggregateAction;

    public WindowAggregateSupplier(String name, WindowInfo windowInfo, Supplier<OV> initAction, AggregateAction<K, V, OV> aggregateAction) {
        this.name = name;
        this.windowInfo = windowInfo;
        this.initAction = initAction;
        this.aggregateAction = aggregateAction;
    }

    @Override
    public Processor<V> 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<V> {
        private final WindowInfo windowInfo;
        private String name;
        private Supplier<OV> initAction;
        private AggregateAction<K, V, OV> aggregateAction;
        private MessageQueue stateTopicMessageQueue;
        private WindowStore<K, OV> windowStore;

        private final AtomicReference<Throwable> errorReference = new AtomicReference<>(null);

        public WindowAggregateProcessor(String name, WindowInfo windowInfo, Supplier<OV> initAction, AggregateAction<K, V, OV> 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<V> 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<Window>
            List<Window> 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<K, OV> 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<K, OV> state = new WindowState<>(key, newValue, time);
                this.windowStore.put(stateTopicMessageQueue, windowKey, state);
                this.idleWindowScaner.putAggregateWindowCallback(windowKey, watermark, this.aggregateWindowFire);
            }

            try {
                List<WindowKey> 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<V> {
        private final String name;
        private final WindowInfo windowInfo;
        private Supplier<OV> initAction;
        private AggregateAction<K, V, OV> aggregateAction;
        private MessageQueue stateTopicMessageQueue;
        private WindowStore<K, OV> windowStore;


        public SessionWindowAggregateProcessor(String name, WindowInfo windowInfo, Supplier<OV> initAction, AggregateAction<K, V, OV> 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<V> 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<Long, Long> newSessionWindowTime = fireIfSessionOut(key, data, time, watermark);

            if (newSessionWindowTime != null) {
                OV oldValue = this.initAction.get();
                OV newValue = this.aggregateAction.calculate(key, data, oldValue);

                WindowState<K, OV> 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<Long/*sessionBegin*/, Long/*sessionEnd*/> fireIfSessionOut(K key, V data, long dataTime, long watermark) throws Throwable {
            List<Pair<WindowKey, WindowState<K, OV>>> 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<Pair<WindowKey, WindowState<K, OV>>> iterator = pairs.iterator();
            int count = 0;
            long lastStateSessionEnd = 0;
            long maxFireSessionEnd = Long.MIN_VALUE;

            while (iterator.hasNext()) {
                Pair<WindowKey, WindowState<K, OV>> 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<WindowKey> 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<WindowKey, WindowState<K, OV>> pair = pairs.get(i);

                WindowKey windowKey = pair.getKey();
                WindowState<K, OV> 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<K, V> {
    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 <NK> Data<NK, V> key(NK key) {
        return new Data<>(key, value, timestamp, new Properties(this.header));
    }

    public <NV> Data<K, NV> 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<K, V> {

    GroupedStream<K, Integer> count();

    <OUT> GroupedStream<K, Integer> count(SelectAction<OUT, V> selectAction);


    GroupedStream<K, V> min(SelectAction<? extends Number, V> selectAction);


    GroupedStream<K, V> max(SelectAction<? extends Number, V> selectAction);


    GroupedStream<K, ? extends Number> sum(SelectAction<? extends Number, V> selectAction);


    GroupedStream<K, V> filter(FilterAction<V> predictor);

    <OUT> GroupedStream<K, OUT> map(ValueMapperAction<V, OUT> valueMapperAction);

    <VR> RStream<VR> flatMap(ValueMapperAction<V,? extends Iterable<? extends VR>> valueMapperAction);


    <OUT> GroupedStream<K, OUT> aggregate(Accumulator<V, OUT> accumulator);

    WindowStream<K, V> window(WindowInfo windowInfo);

    GroupedStream<K, V> addGraphNode(String name, Supplier<Processor<V>> supplier);

    RStream<V> toRStream();

    void sink(String topicName, KeyValueSerializer<K, V> 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<K, V> implements GroupedStream<K, V> {
    private final Pipeline pipeline;
    private final GraphNode parent;

    public GroupedStreamImpl(Pipeline pipeline, GraphNode parent) {
        this.pipeline = pipeline;
        this.parent = parent;
    }

    @Override
    public GroupedStream<K, Integer> count() {
        String name = OperatorNameMaker.makeName(COUNT_PREFIX, pipeline.getJobId());

        Supplier<Processor<V>> 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 <OUT> GroupedStream<K, Integer> count(SelectAction<OUT, V> selectAction) {
        String name = OperatorNameMaker.makeName(COUNT_PREFIX, pipeline.getJobId());

        Supplier<Processor<V>> 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<K, V> min(SelectAction<? extends Number, V> selectAction) {
        String name = OperatorNameMaker.makeName(MIN_PREFIX, pipeline.getJobId());

        Supplier<Processor<V>> supplier = new AggregateSupplier<>(name, parent.getName(), () -> null, (AggregateAction<K, V, V>) (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<K, V> max(SelectAction<? extends Number, V> selectAction) {
        String name = OperatorNameMaker.makeName(MAX_PREFIX, pipeline.getJobId());
        Supplier<Processor<V>> supplier = new AggregateSupplier<>(name, parent.getName(), () -> null, (AggregateAction<K, V, V>) (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<K, ? extends Number> sum(SelectAction<? extends Number, V> selectAction) {
        String name = OperatorNameMaker.makeName(SUM_PREFIX, pipeline.getJobId());
        Supplier<Processor<V>> 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<K, V> filter(FilterAction<V> predictor) {
        String name = OperatorNameMaker.makeName(FILTER_PREFIX, pipeline.getJobId());

        FilterSupplier<V> supplier = new FilterSupplier<>(predictor);
        GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier);

        return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent);
    }

    @Override
    public <OUT> GroupedStream<K, OUT> map(ValueMapperAction<V, OUT> mapperAction) {
        String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId());

        ValueChangeSupplier<V, OUT> supplier = new ValueChangeSupplier<>(mapperAction);
        GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier);

        return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent);
    }

    @Override
    public <VR> RStream<VR> flatMap(ValueMapperAction<V, ? extends Iterable<? extends VR>> valueMapperAction) {
        String name = OperatorNameMaker.makeName(FLAT_MAP_PREFIX, pipeline.getJobId());

        MultiValueChangeSupplier<V, VR> changeSupplier = new MultiValueChangeSupplier<>(valueMapperAction);
        GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), changeSupplier);

        return this.pipeline.addRStreamVirtualNode(graphNode, parent);
    }

    @Override
    public <OUT> GroupedStream<K, OUT> aggregate(Accumulator<V, OUT> accumulator) {
        String name = OperatorNameMaker.makeName(ACCUMULATE_PREFIX, pipeline.getJobId());
        Supplier<Processor<V>> 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<K, V> window(WindowInfo windowInfo) {
        //需要在window里面shuffle
        String name = OperatorNameMaker.makeName(WINDOW_PREFIX, pipeline.getJobId());

        ProcessorNode<V> 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<K, V> addGraphNode(String name, Supplier<Processor<V>> 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<V> toRStream() {
        return new RStreamImpl<>(this.pipeline, parent);
    }

    @Override
    public void sink(String topicName, KeyValueSerializer<K, V> serializer) {
        String name = OperatorNameMaker.makeName(SINK_PREFIX, pipeline.getJobId());

        SinkSupplier<K, V> 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<V1, V2> {
    private RStream<V1> leftStream;
    private RStream<V2> rightStream;
    private JoinType joinType;

    public JoinedStream(RStream<V1> leftStream, RStream<V2> rightStream, JoinType joinType) {
        this.leftStream = leftStream;
        this.rightStream = rightStream;
        this.joinType = joinType;
    }

    public <K> Where<K> where(SelectAction<K, V1> rightSelectAction) {
        return new Where<>(rightSelectAction);
    }

    public class Where<K> {
        private SelectAction<K, V1> leftSelectAction;
        private SelectAction<K, V2> rightSelectAction;

        public Where(SelectAction<K, V1> leftSelectAction) {
            this.leftSelectAction = leftSelectAction;
        }


        public Where<K> equalTo(SelectAction<K, V2> rightSelectAction) {
            this.rightSelectAction = rightSelectAction;
            return this;
        }

        public <OUT> RStream<OUT> apply(ValueJoinAction<V1, V2, OUT> joinAction) {
            List<String> temp = new ArrayList<>();
            Pipeline leftStreamPipeline = JoinedStream.this.leftStream.getPipeline();
            String jobId = leftStreamPipeline.getJobId();

            String name = OperatorNameMaker.makeName(OperatorNameMaker.JOIN_PREFIX, jobId);
            Supplier<Processor<? super OUT>> supplier = new JoinAggregateSupplier<>(name, joinType, joinAction);
            ProcessorNode<OUT> commChild = new ProcessorNode(name, temp, supplier);


            {
                GroupedStream<K, V1> 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<K, V2> 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<K> window(WindowInfo windowInfo) {
            return new JoinWindow<>(this.leftSelectAction, this.rightSelectAction, windowInfo);
        }
    }

    public class JoinWindow<K> {
        private SelectAction<K, V1> leftSelectAction;
        private SelectAction<K, V2> rightSelectAction;
        private WindowInfo windowInfo;

        public JoinWindow(SelectAction<K, V1> leftSelectAction, SelectAction<K, V2> rightSelectAction, WindowInfo windowInfo) {
            this.leftSelectAction = leftSelectAction;
            this.rightSelectAction = rightSelectAction;
            this.windowInfo = windowInfo;
        }

        public <OUT> RStream<OUT> apply(ValueJoinAction<V1, V2, OUT> joinAction) {
            List<String> 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<Processor<? super OUT>> supplier = new JoinWindowAggregateSupplier<>(name, windowInfo, joinAction);
            ProcessorNode<OUT> commChild = new ProcessorNode(name, temp, supplier);


            {
                GroupedStream<K, V1> 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<K, V2> 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<GraphNode> 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 <T> RStream<T> addVirtualSource(GraphNode sourceGraphNode) {
        root.addChild(sourceGraphNode);
        virtualNodes.add(sourceGraphNode);

        return new RStreamImpl<>(this, sourceGraphNode);
    }

    public <T> RStream<T> 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 <K, V> GroupedStreamImpl<K, V> 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 <K, V> WindowStream<K, V> 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<T> {
    RStream<T> selectTimestamp(ValueMapperAction<T, Long> timestampSelector);

    <O> RStream<O> map(ValueMapperAction<T, O> mapperAction);

    <VR> RStream<VR> flatMap(final ValueMapperAction<T, ? extends Iterable<? extends VR>> mapper);

    RStream<T> filter(FilterAction<T> predictor);

    <K> GroupedStream<K, T> keyBy(SelectAction<K, T> selectAction);

    void print();

    RStream<T> foreach(ForeachAction<T> foreachAction);

    <T2> JoinedStream<T, T2> join(RStream<T2> rightStream);

    <T2> JoinedStream<T, T2> leftJoin(RStream<T2> rightStream);

    Pipeline getPipeline();

    void sink(String topicName, KeyValueSerializer<Object, T> 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<T> implements RStream<T> {
    private final Pipeline pipeline;
    private final GraphNode parent;

    public RStreamImpl(Pipeline pipeline, GraphNode parent) {
        this.pipeline = pipeline;
        this.parent = parent;
    }

    @Override
    public RStream<T> selectTimestamp(ValueMapperAction<T, Long> timestampSelector) {
        String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId());

        TimestampSelectorSupplier<T> supplier = new TimestampSelectorSupplier<>(timestampSelector);
        GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), supplier);

        return pipeline.addRStreamVirtualNode(processorNode, parent);
    }

    @Override
    public <O> RStream<O> map(ValueMapperAction<T, O> mapperAction) {
        String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId());

        ValueChangeSupplier<T, O> supplier = new ValueChangeSupplier<>(mapperAction);
        GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), supplier);

        return pipeline.addRStreamVirtualNode(processorNode, parent);
    }

    @Override
    public <VR> RStream<VR> flatMap(ValueMapperAction<T, ? extends Iterable<? extends VR>> mapper) {
        String name = OperatorNameMaker.makeName(FLAT_MAP_PREFIX, pipeline.getJobId());

        MultiValueChangeSupplier<T, VR> changeSupplier = new MultiValueChangeSupplier<>(mapper);
        GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), changeSupplier);

        return pipeline.addRStreamVirtualNode(processorNode, parent);
    }

    @Override
    public RStream<T> filter(FilterAction<T> predictor) {
        String name = OperatorNameMaker.makeName(FILTER_PREFIX, pipeline.getJobId());

        FilterSupplier<T> supplier = new FilterSupplier<>(predictor);
        GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), supplier);

        return pipeline.addRStreamVirtualNode(processorNode, parent);
    }

    @Override
    public <K> GroupedStream<K, T> keyBy(SelectAction<K, T> selectAction) {
        String name = OperatorNameMaker.makeName(GROUPBY_PREFIX, pipeline.getJobId());

        KeySelectSupplier<K, T> 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<T> printSupplier = new PrintSupplier<>();
        GraphNode sinkGraphNode = new SinkGraphNode<>(name, parent.getName(), null, printSupplier);

        pipeline.addVirtualSink(sinkGraphNode, parent);
    }

    @Override
    public RStream<T> foreach(ForeachAction<T> foreachAction) {
        String name = OperatorNameMaker.makeName(FOR_EACH_PREFIX, pipeline.getJobId());

        ForeachSupplier<T> supplier = new ForeachSupplier<T>(foreachAction);

        ProcessorNode<T> node = new ProcessorNode<>(name, parent.getName(), supplier);

        return pipeline.addRStreamVirtualNode(node, parent);
    }

    @Override
    public <T2> JoinedStream<T, T2> join(RStream<T2> rightStream) {
        return new JoinedStream<>(this, rightStream, JoinType.INNER_JOIN);
    }

    @Override
    public <T2> JoinedStream<T, T2> leftJoin(RStream<T2> rightStream) {
        return new JoinedStream<>(this, rightStream, JoinType.LEFT_JOIN);
    }

    @Override
    public Pipeline getPipeline() {
        return pipeline;
    }

    @Override
    public void sink(String topicName, KeyValueSerializer<Object, T> serializer) {
        String name = OperatorNameMaker.makeName(SINK_PREFIX, pipeline.getJobId());

        SinkSupplier<Object, T> 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<Pipeline> 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 <OUT> RStream<OUT> source(String topicName, KeyValueDeserializer<Void, OUT> 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<GraphNode> 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 <K> key type
 * @param <V> value type
 */
public interface WindowStream<K, V> {
    WindowStream<K, Integer> count();

    WindowStream<K, Double> avg();

    WindowStream<K, V> min(SelectAction<? extends Number, V> selectAction);

    WindowStream<K, V> max(SelectAction<? extends Number, V> selectAction);

    WindowStream<K, ? extends Number> sum(SelectAction<? extends Number, V> selectAction);

    WindowStream<K, V> filter(FilterAction<V> predictor);

    <OUT> WindowStream<K, OUT> map(ValueMapperAction<V, OUT> mapperAction);

    <OUT> WindowStream<K, OUT> aggregate(AggregateAction<K, V, OUT> aggregateAction);

    <OUT> WindowStream<K, OUT> aggregate(Accumulator<V, OUT> accumulator);

    RStream<V> toRStream();

    void sink(String topicName, KeyValueSerializer<K, V> 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<K, V> implements WindowStream<K, V> {
    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<K, Integer> count() {
        String name = OperatorNameMaker.makeName(COUNT_PREFIX, pipeline.getJobId());
        Supplier<Processor<V>> supplier = new WindowAccumulatorSupplier<>(name, windowInfo, value -> value, new CountAccumulator<>());

        //是否需要分组计算
        ProcessorNode<V> 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<K, Double> avg() {
        String name = OperatorNameMaker.makeName(WINDOW_AVG_PREFIX, pipeline.getJobId());
        Supplier<Processor<V>> supplier = new WindowAccumulatorSupplier<>(name, windowInfo, value -> value, new AvgAccumulator<>());

        //是否需要分组计算
        ProcessorNode<V> 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<K, V> min(SelectAction<? extends Number, V> selectAction) {
        String name = OperatorNameMaker.makeName(MIN_PREFIX, pipeline.getJobId());

        Supplier<Processor<V>> supplier = new WindowAggregateSupplier<>(name, windowInfo, () -> null, (AggregateAction<K, V, V>) (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<K, V> max(SelectAction<? extends Number, V> selectAction) {
        String name = OperatorNameMaker.makeName(MAX_PREFIX, pipeline.getJobId());

        Supplier<Processor<V>> supplier = new WindowAggregateSupplier<>(name, windowInfo, () -> null, (AggregateAction<K, V, V>) (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<K, ? extends Number> sum(SelectAction<? extends Number, V> selectAction) {
        String name = OperatorNameMaker.makeName(SUM_PREFIX, pipeline.getJobId());
        Supplier<Processor<V>> 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<K, V> filter(FilterAction<V> predictor) {
        String name = OperatorNameMaker.makeName(FILTER_PREFIX, pipeline.getJobId());

        FilterSupplier<V> supplier = new FilterSupplier<>(predictor);
        GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier);

        return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo);
    }

    @Override
    public <OUT> WindowStream<K, OUT> map(ValueMapperAction<V, OUT> mapperAction) {
        String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId());

        ValueChangeSupplier<V, OUT> supplier = new ValueChangeSupplier<>(mapperAction);
        GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier);

        return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo);
    }

    @Override
    public <OUT> WindowStream<K, OUT> aggregate(AggregateAction<K, V, OUT> aggregateAction) {
        String name = OperatorNameMaker.makeName(AGGREGATE_PREFIX, pipeline.getJobId());

        Supplier<Processor<V>> supplier = new WindowAggregateSupplier<>(name, windowInfo, () -> null, aggregateAction);

        //是否需要分组计算
        ProcessorNode<V> 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 <OUT> WindowStream<K, OUT> aggregate(Accumulator<V, OUT> accumulator) {
        String name = OperatorNameMaker.makeName(AGGREGATE_PREFIX, pipeline.getJobId());

        Supplier<Processor<V>> supplier = new WindowAccumulatorSupplier<>(name, windowInfo, value -> value, accumulator);

        //是否需要分组计算
        ProcessorNode<V> 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<V> toRStream() {
        return new RStreamImpl<>(this.pipeline, parent);
    }

    @Override
    public void sink(String topicName, KeyValueSerializer<K, V> serializer) {
        String name = OperatorNameMaker.makeName(SINK_PREFIX, pipeline.getJobId());

        SinkSupplier<K, V> 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<T> implements Processor<T> {
    private final List<Processor<T>> children = new ArrayList<>();
    protected StreamContext<T> context;
    protected long allowDelay = 0;

    @Override
    public void addChild(Processor<T> processor) {
        children.add(processor);
    }

    @Override
    public void preProcess(StreamContext<T> 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<Processor<T>> 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 <KEY> Data<KEY, T> convert(Data<?, ?> data) {
        return (Data<KEY, T>) new Data<>(data.getKey(), data.getValue(), data.getTimestamp(), data.getHeader());
    }



    private final ByteBuf buf = Unpooled.buffer(16);
    /**
     * encode
     * <pre>
     * +-----------+---------------+-------------+-------------+
     * | Int(4)    |   className  | Int(4)       | value bytes |
     * | classname |              |object length |             |
     * +-----------+--------------+---------------+-------------+
     * </pre>
     * @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
     * <pre>
     * +-----------+---------------+-------------+-------------+
     * | Int(4)    |   className  | Int(4)       | value bytes |
     * | classname |              |object length |             |
     * +-----------+--------------+---------------+-------------+
     * </pre>
     * @param bytes the byte array to deserialize;
     * @return V the result of deserialize
     * @throws Throwable deserialize exception.
     */
    @SuppressWarnings("unchecked")
    public <V> 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<V> clazz = (Class<V>)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 or
Download .txt
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
Download .txt
SYMBOL INDEX (803 symbols across 121 files)

FILE: core/src/main/java/org/apache/rocketmq/streams/core/RocketMQStream.java
  class RocketMQStream (line 35) | public class RocketMQStream {
    method RocketMQStream (line 47) | public RocketMQStream(TopologyBuilder topologyBuilder, Properties prop...
    method start (line 53) | public synchronized void start() {
    method stop (line 79) | public void stop() {
    method isRunning (line 87) | public boolean isRunning() {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/common/Constant.java
  class Constant (line 21) | public class Constant {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/DataProcessThrowable.java
  class DataProcessThrowable (line 19) | public class DataProcessThrowable extends Throwable {
    method DataProcessThrowable (line 20) | public DataProcessThrowable() {
    method DataProcessThrowable (line 23) | public DataProcessThrowable(String message) {
    method DataProcessThrowable (line 27) | public DataProcessThrowable(String message, Throwable cause) {
    method DataProcessThrowable (line 31) | public DataProcessThrowable(Throwable cause) {
    method DataProcessThrowable (line 35) | public DataProcessThrowable(String message, Throwable cause, boolean e...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/DeserializeThrowable.java
  class DeserializeThrowable (line 19) | public class DeserializeThrowable extends Throwable {
    method DeserializeThrowable (line 22) | public DeserializeThrowable() {
    method DeserializeThrowable (line 25) | public DeserializeThrowable(String message) {
    method DeserializeThrowable (line 29) | public DeserializeThrowable(String message, Throwable cause) {
    method DeserializeThrowable (line 33) | public DeserializeThrowable(Throwable cause) {
    method DeserializeThrowable (line 37) | public DeserializeThrowable(String message, Throwable cause, boolean e...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/RStreamsException.java
  class RStreamsException (line 19) | public class RStreamsException extends RuntimeException {
    method RStreamsException (line 22) | public RStreamsException() {
    method RStreamsException (line 25) | public RStreamsException(String message) {
    method RStreamsException (line 29) | public RStreamsException(String message, Throwable cause) {
    method RStreamsException (line 33) | public RStreamsException(Throwable cause) {
    method RStreamsException (line 37) | public RStreamsException(String message, Throwable cause, boolean enab...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/RecoverStateStoreThrowable.java
  class RecoverStateStoreThrowable (line 19) | public class RecoverStateStoreThrowable extends Throwable {
    method RecoverStateStoreThrowable (line 20) | public RecoverStateStoreThrowable() {
    method RecoverStateStoreThrowable (line 23) | public RecoverStateStoreThrowable(String message) {
    method RecoverStateStoreThrowable (line 27) | public RecoverStateStoreThrowable(String message, Throwable cause) {
    method RecoverStateStoreThrowable (line 31) | public RecoverStateStoreThrowable(Throwable cause) {
    method RecoverStateStoreThrowable (line 35) | public RecoverStateStoreThrowable(String message, Throwable cause, boo...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/AggregateAction.java
  type AggregateAction (line 19) | public interface AggregateAction<K, V, OUT> {
    method calculate (line 20) | OUT calculate(K key, V value, OUT accumulator);

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/FilterAction.java
  type FilterAction (line 19) | @FunctionalInterface
    method apply (line 26) | boolean apply(final T value);

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/ForeachAction.java
  type ForeachAction (line 19) | @FunctionalInterface
    method apply (line 21) | void apply(T value);

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/SelectAction.java
  type SelectAction (line 20) | @FunctionalInterface
    method select (line 22) | OUT select(IN value);

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/ValueJoinAction.java
  type ValueJoinAction (line 20) | public interface ValueJoinAction<V1, V2, OUT> {
    method apply (line 22) | OUT apply(final V1 value1, final V2 value2);

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/ValueMapperAction.java
  type ValueMapperAction (line 19) | public interface ValueMapperAction<T, O> {
    method convert (line 20) | O convert(final T value) throws Throwable;

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/Accumulator.java
  type Accumulator (line 22) | public interface Accumulator<V, R> {
    method addValue (line 23) | void addValue(V value);
    method merge (line 25) | void merge(Accumulator<V, R> other);
    method result (line 33) | R result(Properties context);
    method clone (line 35) | Accumulator<V, R> clone();

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/AvgAccumulator.java
  class AvgAccumulator (line 21) | public class AvgAccumulator<V> implements Accumulator<V, Double> {
    method addValue (line 25) | @Override
    method merge (line 43) | @Override
    method result (line 57) | @Override
    method clone (line 62) | @Override
    method getAvg (line 67) | public Double getAvg() {
    method setAvg (line 71) | public void setAvg(Double avg) {
    method getNum (line 75) | public Integer getNum() {
    method setNum (line 79) | public void setNum(Integer num) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/CountAccumulator.java
  class CountAccumulator (line 22) | public class CountAccumulator<V> implements Accumulator<V, Integer> {
    method addValue (line 25) | @Override
    method merge (line 30) | @Override
    method result (line 35) | @Override
    method getCount (line 40) | public Integer getCount() {
    method setCount (line 44) | public void setCount(Integer count) {
    method clone (line 48) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/MinAccumulator.java
  class MinAccumulator (line 21) | public class MinAccumulator<V> implements Accumulator<V, Double> {
    method addValue (line 24) | @Override
    method merge (line 39) | @Override
    method result (line 44) | @Override
    method clone (line 49) | @Override
    method getMin (line 54) | public Number getMin() {
    method setMin (line 58) | public void setMin(Number min) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AccumulatorSupplier.java
  class AccumulatorSupplier (line 33) | public class AccumulatorSupplier<K, V, R, OV> implements Supplier<Proces...
    method AccumulatorSupplier (line 39) | public AccumulatorSupplier(String currentName, String parentName, Sele...
    method get (line 46) | @Override
    class AccumulatorProcessor (line 51) | private class AccumulatorProcessor extends AbstractProcessor<V> {
      method AccumulatorProcessor (line 59) | public AccumulatorProcessor(String currentName, String parentName, S...
      method preProcess (line 66) | @Override
      method process (line 75) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AddTagSupplier.java
  class AddTagSupplier (line 27) | public class AddTagSupplier<T> implements Supplier<Processor<T>> {
    method AddTagSupplier (line 30) | public AddTagSupplier(Supplier<Object> value) {
    method AddTagSupplier (line 35) | public AddTagSupplier() {
    method get (line 38) | @Override
    class AddTagProcessor (line 43) | class AddTagProcessor extends AbstractProcessor<T> {
      method AddTagProcessor (line 46) | public AddTagProcessor(Properties properties) {
      method process (line 50) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AggregateSupplier.java
  class AggregateSupplier (line 32) | public class AggregateSupplier<K, V, OV> implements Supplier<Processor<V...
    method AggregateSupplier (line 38) | public AggregateSupplier(String currentName, String parentName, Suppli...
    method get (line 46) | @Override
    class AggregateProcessor (line 51) | private class AggregateProcessor extends AbstractProcessor<V> {
      method AggregateProcessor (line 59) | public AggregateProcessor(String currentName, String parentName, Sup...
      method preProcess (line 67) | @Override
      method process (line 76) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/FilterSupplier.java
  class FilterSupplier (line 26) | public class FilterSupplier<T> implements Supplier<Processor<T>> {
    method FilterSupplier (line 29) | public FilterSupplier(FilterAction<T> filterAction) {
    method get (line 33) | @Override
    class FilterProcessor (line 38) | private class FilterProcessor extends AbstractProcessor<T> {
      method FilterProcessor (line 41) | public FilterProcessor(FilterAction<T> filterAction) {
      method process (line 45) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/ForeachSupplier.java
  class ForeachSupplier (line 26) | public class ForeachSupplier<T> implements Supplier<Processor<T>> {
    method ForeachSupplier (line 29) | public ForeachSupplier(ForeachAction<T> foreachAction) {
    method get (line 33) | @Override
    class ForeachProcessor (line 38) | class ForeachProcessor extends AbstractProcessor<T> {
      method ForeachProcessor (line 41) | public ForeachProcessor(ForeachAction<T> foreachAction) {
      method process (line 45) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/JoinAggregateSupplier.java
  class JoinAggregateSupplier (line 38) | public class JoinAggregateSupplier<K, V1, V2, OUT> implements Supplier<P...
    method JoinAggregateSupplier (line 45) | public JoinAggregateSupplier(String name, JoinType joinType, ValueJoin...
    method get (line 51) | @Override
    class JoinStreamAggregateProcessor (line 56) | private class JoinStreamAggregateProcessor extends AbstractProcessor<O...
      method JoinStreamAggregateProcessor (line 64) | public JoinStreamAggregateProcessor(String name, JoinType joinType, ...
      method preProcess (line 70) | @Override
      method process (line 78) | @Override
      method store (line 88) | private void store(Object key, Object data, StreamType streamType) t...
      method fire (line 105) | @SuppressWarnings("unchecked")
      method doFire (line 156) | private void doFire(V1 v1Data, V2 v2Data) throws Throwable {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/JoinWindowAggregateSupplier.java
  class JoinWindowAggregateSupplier (line 47) | public class JoinWindowAggregateSupplier<K, V1, V2, OUT> implements Supp...
    method JoinWindowAggregateSupplier (line 55) | public JoinWindowAggregateSupplier(String name, WindowInfo windowInfo,...
    method get (line 62) | @Override
    class JoinStreamWindowAggregateProcessor (line 68) | @SuppressWarnings("unchecked")
      method JoinStreamWindowAggregateProcessor (line 78) | public JoinStreamWindowAggregateProcessor(String name, WindowInfo wi...
      method preProcess (line 85) | @Override
      method process (line 105) | @Override
      method store (line 133) | private void store(Object key, Object data, long time, long watermar...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/KeySelectSupplier.java
  class KeySelectSupplier (line 25) | public class KeySelectSupplier<KEY, T> implements Supplier<Processor<T>> {
    method KeySelectSupplier (line 28) | public KeySelectSupplier(SelectAction<KEY, T> selectAction) {
    method get (line 32) | @Override
    class MapperProcessor (line 37) | private class MapperProcessor extends AbstractProcessor<T> {
      method MapperProcessor (line 41) | public MapperProcessor(SelectAction<KEY, T> selectAction) {
      method process (line 45) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/MultiValueChangeSupplier.java
  class MultiValueChangeSupplier (line 29) | public class MultiValueChangeSupplier<T, VR> implements Supplier<Process...
    method MultiValueChangeSupplier (line 33) | public MultiValueChangeSupplier(ValueMapperAction<T, ? extends Iterabl...
    method get (line 37) | @Override
    class MultiValueMapperProcessor (line 42) | static class MultiValueMapperProcessor<T, VR> extends AbstractProcesso...
      method MultiValueMapperProcessor (line 45) | public MultiValueMapperProcessor(ValueMapperAction<T, ? extends Iter...
      method process (line 49) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/PrintSupplier.java
  class PrintSupplier (line 28) | public class PrintSupplier<T> implements Supplier<Processor<T>> {
    method get (line 31) | @Override
    class PrintProcessor (line 36) | static class PrintProcessor<T> extends AbstractProcessor<T> {
      method PrintProcessor (line 39) | public PrintProcessor() {
      method process (line 43) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SinkSupplier.java
  class SinkSupplier (line 34) | public class SinkSupplier<K, T> implements Supplier<Processor<T>> {
    method SinkSupplier (line 40) | public SinkSupplier(String topicName, KeyValueSerializer<K, T> seriali...
    method get (line 45) | @Override
    class SinkProcessor (line 50) | private class SinkProcessor extends AbstractProcessor<T> {
      method SinkProcessor (line 56) | public SinkProcessor(String topicName, KeyValueSerializer<K, T> seri...
      method preProcess (line 61) | @Override
      method process (line 69) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SourceSupplier.java
  class SourceSupplier (line 29) | public class SourceSupplier<K, V> implements Supplier<Processor<V>> {
    method SourceSupplier (line 33) | public SourceSupplier(String topicName, KeyValueDeserializer<K, V> des...
    method get (line 38) | @Override
    type SourceProcessor (line 43) | public interface SourceProcessor<K, V> extends Processor<V> {
      method deserialize (line 44) | Pair<K, V> deserialize(String keyClass, String valueClass, byte[] da...
      method getTimestamp (line 46) | long getTimestamp(MessageExt originData, TimeType timeType);
    class SourceProcessorImpl (line 49) | private class SourceProcessorImpl extends AbstractProcessor<V> impleme...
      method SourceProcessorImpl (line 52) | public SourceProcessorImpl(KeyValueDeserializer<K, V> deserializer) {
      method deserialize (line 56) | @Override
      method getTimestamp (line 66) | @Override
      method process (line 80) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SumAggregate.java
  class SumAggregate (line 23) | public class SumAggregate<K, V> implements AggregateAction<K, V, Number> {
    method SumAggregate (line 26) | public SumAggregate(SelectAction<? extends Number, V> selectAction) {
    method calculate (line 30) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/TimestampSelectorSupplier.java
  class TimestampSelectorSupplier (line 27) | public class TimestampSelectorSupplier<T> implements Supplier<Processor<...
    method TimestampSelectorSupplier (line 30) | public TimestampSelectorSupplier(ValueMapperAction<T, Long> valueMappe...
    method get (line 34) | @Override
    class TimestampSelector (line 40) | static class TimestampSelector<T> extends AbstractProcessor<T> {
      method TimestampSelector (line 43) | public TimestampSelector(ValueMapperAction<T, Long> valueMapperActio...
      method process (line 47) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/ValueChangeSupplier.java
  class ValueChangeSupplier (line 29) | public class ValueChangeSupplier<T, O> implements Supplier<Processor<T>> {
    method ValueChangeSupplier (line 34) | public ValueChangeSupplier(ValueMapperAction<T, O> valueMapperAction) {
    method get (line 38) | @Override
    class ValueMapperProcessor (line 44) | static class ValueMapperProcessor<T, O> extends AbstractProcessor<T> {
      method ValueMapperProcessor (line 48) | public ValueMapperProcessor(ValueMapperAction<T, O> valueMapperActio...
      method process (line 52) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/WindowAccumulatorSupplier.java
  class WindowAccumulatorSupplier (line 44) | public class WindowAccumulatorSupplier<K, V, R, OV> implements Supplier<...
    method WindowAccumulatorSupplier (line 51) | public WindowAccumulatorSupplier(String name, WindowInfo windowInfo,
    method get (line 59) | @Override
    class WindowAccumulatorProcessor (line 74) | public class WindowAccumulatorProcessor extends AbstractWindowProcesso...
      method WindowAccumulatorProcessor (line 84) | public WindowAccumulatorProcessor(String name, WindowInfo windowInfo...
      method preProcess (line 91) | @Override
      method process (line 112) | @Override
    class SessionWindowAccumulatorProcessor (line 168) | private class SessionWindowAccumulatorProcessor extends AbstractWindow...
      method SessionWindowAccumulatorProcessor (line 176) | public SessionWindowAccumulatorProcessor(String name, WindowInfo win...
      method preProcess (line 183) | @Override
      method process (line 201) | @Override
      method fireIfSessionOut (line 235) | @SuppressWarnings("unchecked")

FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/WindowAggregateSupplier.java
  class WindowAggregateSupplier (line 45) | public class WindowAggregateSupplier<K, V, OV> implements Supplier<Proce...
    method WindowAggregateSupplier (line 52) | public WindowAggregateSupplier(String name, WindowInfo windowInfo, Sup...
    method get (line 59) | @Override
    class WindowAggregateProcessor (line 74) | private class WindowAggregateProcessor extends AbstractWindowProcessor...
      method WindowAggregateProcessor (line 84) | public WindowAggregateProcessor(String name, WindowInfo windowInfo, ...
      method preProcess (line 91) | @Override
      method process (line 112) | @Override
    class SessionWindowAggregateProcessor (line 171) | private class SessionWindowAggregateProcessor extends AbstractWindowPr...
      method SessionWindowAggregateProcessor (line 180) | public SessionWindowAggregateProcessor(String name, WindowInfo windo...
      method preProcess (line 187) | @Override
      method process (line 205) | @Override
      method fireIfSessionOut (line 239) | @SuppressWarnings("unchecked")

FILE: core/src/main/java/org/apache/rocketmq/streams/core/metadata/Data.java
  class Data (line 21) | public class Data<K, V> {
    method Data (line 27) | public Data(K key, V value, Long timestamp, Properties header) {
    method getKey (line 34) | public K getKey() {
    method setKey (line 38) | public void setKey(K key) {
    method getValue (line 42) | public V getValue() {
    method setValue (line 46) | public void setValue(V value) {
    method getTimestamp (line 50) | public Long getTimestamp() {
    method setTimestamp (line 54) | public void setTimestamp(Long timestamp) {
    method getHeader (line 58) | public Properties getHeader() {
    method setHeader (line 62) | public void setHeader(Properties header) {
    method key (line 66) | public <NK> Data<NK, V> key(NK key) {
    method value (line 70) | public <NV> Data<K, NV> value(NV value) {
    method toString (line 74) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/metadata/StreamConfig.java
  class StreamConfig (line 19) | public class StreamConfig {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/GroupedStream.java
  type GroupedStream (line 29) | public interface GroupedStream<K, V> {
    method count (line 31) | GroupedStream<K, Integer> count();
    method count (line 33) | <OUT> GroupedStream<K, Integer> count(SelectAction<OUT, V> selectAction);
    method min (line 36) | GroupedStream<K, V> min(SelectAction<? extends Number, V> selectAction);
    method max (line 39) | GroupedStream<K, V> max(SelectAction<? extends Number, V> selectAction);
    method sum (line 42) | GroupedStream<K, ? extends Number> sum(SelectAction<? extends Number, ...
    method filter (line 45) | GroupedStream<K, V> filter(FilterAction<V> predictor);
    method map (line 47) | <OUT> GroupedStream<K, OUT> map(ValueMapperAction<V, OUT> valueMapperA...
    method flatMap (line 49) | <VR> RStream<VR> flatMap(ValueMapperAction<V,? extends Iterable<? exte...
    method aggregate (line 52) | <OUT> GroupedStream<K, OUT> aggregate(Accumulator<V, OUT> accumulator);
    method window (line 54) | WindowStream<K, V> window(WindowInfo windowInfo);
    method addGraphNode (line 56) | GroupedStream<K, V> addGraphNode(String name, Supplier<Processor<V>> s...
    method toRStream (line 58) | RStream<V> toRStream();
    method sink (line 60) | void sink(String topicName, KeyValueSerializer<K, V> serializer);

FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/GroupedStreamImpl.java
  class GroupedStreamImpl (line 54) | public class GroupedStreamImpl<K, V> implements GroupedStream<K, V> {
    method GroupedStreamImpl (line 58) | public GroupedStreamImpl(Pipeline pipeline, GraphNode parent) {
    method count (line 63) | @Override
    method count (line 79) | @Override
    method min (line 95) | @Override
    method max (line 126) | @Override
    method sum (line 156) | @Override
    method filter (line 171) | @Override
    method map (line 181) | @Override
    method flatMap (line 191) | @Override
    method aggregate (line 201) | @Override
    method window (line 216) | @Override
    method addGraphNode (line 234) | @Override
    method toRStream (line 246) | @Override
    method sink (line 251) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/JoinedStream.java
  class JoinedStream (line 39) | public class JoinedStream<V1, V2> {
    method JoinedStream (line 44) | public JoinedStream(RStream<V1> leftStream, RStream<V2> rightStream, J...
    method where (line 50) | public <K> Where<K> where(SelectAction<K, V1> rightSelectAction) {
    class Where (line 54) | public class Where<K> {
      method Where (line 58) | public Where(SelectAction<K, V1> leftSelectAction) {
      method equalTo (line 63) | public Where<K> equalTo(SelectAction<K, V2> rightSelectAction) {
      method apply (line 68) | public <OUT> RStream<OUT> apply(ValueJoinAction<V1, V2, OUT> joinAct...
      method window (line 108) | public JoinWindow<K> window(WindowInfo windowInfo) {
    class JoinWindow (line 113) | public class JoinWindow<K> {
      method JoinWindow (line 118) | public JoinWindow(SelectAction<K, V1> leftSelectAction, SelectAction...
      method apply (line 124) | public <OUT> RStream<OUT> apply(ValueJoinAction<V1, V2, OUT> joinAct...
      method copy (line 179) | private WindowInfo copy(WindowInfo windowInfo) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/Pipeline.java
  class Pipeline (line 27) | public class Pipeline {
    method addRealNode (line 31) | @Override
    method Pipeline (line 37) | public Pipeline(String jobId) {
    method addVirtualSource (line 41) | public <T> RStream<T> addVirtualSource(GraphNode sourceGraphNode) {
    method addRStreamVirtualNode (line 48) | public <T> RStream<T> addRStreamVirtualNode(GraphNode currentNode, Gra...
    method addGroupedStreamVirtualNode (line 60) | public <K, V> GroupedStreamImpl<K, V> addGroupedStreamVirtualNode(Grap...
    method addWindowStreamVirtualNode (line 72) | public <K, V> WindowStream<K, V> addWindowStreamVirtualNode(GraphNode ...
    method addVirtualSink (line 84) | public void addVirtualSink(GraphNode currentNode, GraphNode parentNode) {
    method getRoot (line 92) | public GraphNode getRoot() {
    method getLastNode (line 97) | public GraphNode getLastNode() {
    method getJobId (line 101) | public String getJobId() {
    method getVirtualNodesNum (line 105) | public int getVirtualNodesNum() {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/RStream.java
  type RStream (line 25) | public interface RStream<T> {
    method selectTimestamp (line 26) | RStream<T> selectTimestamp(ValueMapperAction<T, Long> timestampSelector);
    method map (line 28) | <O> RStream<O> map(ValueMapperAction<T, O> mapperAction);
    method flatMap (line 30) | <VR> RStream<VR> flatMap(final ValueMapperAction<T, ? extends Iterable...
    method filter (line 32) | RStream<T> filter(FilterAction<T> predictor);
    method keyBy (line 34) | <K> GroupedStream<K, T> keyBy(SelectAction<K, T> selectAction);
    method print (line 36) | void print();
    method foreach (line 38) | RStream<T> foreach(ForeachAction<T> foreachAction);
    method join (line 40) | <T2> JoinedStream<T, T2> join(RStream<T2> rightStream);
    method leftJoin (line 42) | <T2> JoinedStream<T, T2> leftJoin(RStream<T2> rightStream);
    method getPipeline (line 44) | Pipeline getPipeline();
    method sink (line 46) | void sink(String topicName, KeyValueSerializer<Object, T> serializer);

FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/RStreamImpl.java
  class RStreamImpl (line 48) | public class RStreamImpl<T> implements RStream<T> {
    method RStreamImpl (line 52) | public RStreamImpl(Pipeline pipeline, GraphNode parent) {
    method selectTimestamp (line 57) | @Override
    method map (line 67) | @Override
    method flatMap (line 77) | @Override
    method filter (line 87) | @Override
    method keyBy (line 97) | @Override
    method print (line 108) | @Override
    method foreach (line 118) | @Override
    method join (line 129) | @Override
    method leftJoin (line 134) | @Override
    method getPipeline (line 139) | @Override
    method sink (line 144) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/StreamBuilder.java
  class StreamBuilder (line 31) | public class StreamBuilder {
    method StreamBuilder (line 36) | public StreamBuilder(String jobId) throws Throwable {
    method checkJobId (line 42) | private void checkJobId(String jobId) {
    method source (line 48) | public <OUT> RStream<OUT> source(String topicName, KeyValueDeserialize...
    method build (line 59) | public TopologyBuilder build() {
    method doBuild (line 69) | private void doBuild(GraphNode graphNode) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/WindowStream.java
  type WindowStream (line 31) | public interface WindowStream<K, V> {
    method count (line 32) | WindowStream<K, Integer> count();
    method avg (line 34) | WindowStream<K, Double> avg();
    method min (line 36) | WindowStream<K, V> min(SelectAction<? extends Number, V> selectAction);
    method max (line 38) | WindowStream<K, V> max(SelectAction<? extends Number, V> selectAction);
    method sum (line 40) | WindowStream<K, ? extends Number> sum(SelectAction<? extends Number, V...
    method filter (line 42) | WindowStream<K, V> filter(FilterAction<V> predictor);
    method map (line 44) | <OUT> WindowStream<K, OUT> map(ValueMapperAction<V, OUT> mapperAction);
    method aggregate (line 46) | <OUT> WindowStream<K, OUT> aggregate(AggregateAction<K, V, OUT> aggreg...
    method aggregate (line 48) | <OUT> WindowStream<K, OUT> aggregate(Accumulator<V, OUT> accumulator);
    method toRStream (line 50) | RStream<V> toRStream();
    method sink (line 52) | void sink(String topicName, KeyValueSerializer<K, V> serializer);

FILE: core/src/main/java/org/apache/rocketmq/streams/core/rstream/WindowStreamImpl.java
  class WindowStreamImpl (line 54) | public class WindowStreamImpl<K, V> implements WindowStream<K, V> {
    method WindowStreamImpl (line 59) | public WindowStreamImpl(Pipeline pipeline, GraphNode parent, WindowInf...
    method count (line 65) | @Override
    method avg (line 81) | @Override
    method min (line 97) | @Override public WindowStream<K, V> min(SelectAction<? extends Number,...
    method max (line 127) | @Override public WindowStream<K, V> max(SelectAction<? extends Number,...
    method sum (line 157) | @Override public WindowStream<K, ? extends Number> sum(SelectAction<? ...
    method filter (line 171) | @Override
    method map (line 181) | @Override
    method aggregate (line 191) | @Override
    method aggregate (line 209) | @Override
    method toRStream (line 227) | @Override
    method sink (line 232) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/AbstractProcessor.java
  class AbstractProcessor (line 37) | public abstract class AbstractProcessor<T> implements Processor<T> {
    method addChild (line 42) | @Override
    method preProcess (line 47) | @Override
    method getChildren (line 58) | protected List<Processor<T>> getChildren() {
    method waitStateReplay (line 62) | protected StateStore waitStateReplay() throws RecoverStateStoreThrowab...
    method convert (line 70) | @SuppressWarnings("unchecked")
    method object2Byte (line 90) | protected byte[] object2Byte(Object obj) throws JsonProcessingException {
    method byte2Object (line 125) | @SuppressWarnings("unchecked")
    method toHexString (line 153) | protected String toHexString(Object source) throws JsonProcessingExcep...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/AbstractWindowProcessor.java
  class AbstractWindowProcessor (line 38) | public abstract class AbstractWindowProcessor<V> extends AbstractProcess...
    method calculateWindow (line 48) | protected List<Window> calculateWindow(WindowInfo windowInfo, long val...
    method watermark (line 64) | protected long watermark(long watermark, MessageQueue stateTopicMessag...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/MessageQueueListenerWrapper.java
  class MessageQueueListenerWrapper (line 35) | class MessageQueueListenerWrapper implements MessageQueueListener {
    method MessageQueueListenerWrapper (line 46) | MessageQueueListenerWrapper(MessageQueueListener originListener, Topol...
    method messageQueueChanged (line 51) | @Override
    method buildTask (line 81) | private void buildTask(Set<MessageQueue> addQueues) {
    method removeTask (line 91) | private void removeTask(Set<MessageQueue> removeQueues) {
    method selectProcessor (line 98) | @SuppressWarnings("unchecked")
    method setRecoverHandler (line 103) | public void setRecoverHandler(BiFunction<Set<MessageQueue>, Set<Messag...
    method setResetOffsetHandler (line 107) | public void setResetOffsetHandler(Function<Set<MessageQueue>, Throwabl...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/Processor.java
  type Processor (line 21) | public interface Processor<T> {
    method addChild (line 22) | void addChild(Processor<T> processor);
    method preProcess (line 25) | void preProcess(StreamContext<T> context) throws RecoverStateStoreThro...
    method process (line 27) | void process(T data) throws Throwable;

FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/RocketMQClient.java
  class RocketMQClient (line 40) | public class RocketMQClient {
    method RocketMQClient (line 44) | public RocketMQClient(String nameSrvAddr) {
    method pullConsumer (line 48) | public DefaultLitePullConsumer pullConsumer(String groupName,
    method producer (line 63) | public DefaultMQProducer producer(String groupName) {
    method getMQAdmin (line 69) | public DefaultMQAdminExt getMQAdmin() throws MQClientException {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/StreamContext.java
  type StreamContext (line 27) | public interface StreamContext<V> {
    method init (line 28) | void init(List<Processor<V>> childrenProcessors);
    method getStateStore (line 30) | StateStore getStateStore();
    method getDataTime (line 32) | long getDataTime();
    method getKey (line 34) | <K> K getKey();
    method getUserProperties (line 36) | Properties getUserProperties();
    method getHeader (line 38) | Properties getHeader();
    method getDefaultMQProducer (line 40) | DefaultMQProducer getDefaultMQProducer();
    method getSourceBrokerName (line 42) | String getSourceBrokerName();
    method getSourceTopic (line 44) | String getSourceTopic();
    method getSourceQueueId (line 46) | Integer getSourceQueueId();
    method getDefaultWindowScaner (line 48) | IdleWindowScaner getDefaultWindowScaner();
    method copy (line 50) | StreamContext<V> copy();
    method forward (line 52) | <K> void forward(Data<K, V> data) throws Throwable;

FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/StreamContextImpl.java
  class StreamContextImpl (line 37) | public class StreamContextImpl<V> implements StreamContext<V> {
    method StreamContextImpl (line 53) | StreamContextImpl(Properties properties,
    method init (line 67) | @Override
    method getStateStore (line 75) | @Override
    method getDefaultMQProducer (line 80) | @Override
    method getSourceBrokerName (line 86) | public String getSourceBrokerName() {
    method getSourceTopic (line 91) | public String getSourceTopic() {
    method getSourceQueueId (line 96) | public Integer getSourceQueueId() {
    method getDataTime (line 101) | @Override
    method getKey (line 107) | @Override
    method setKey (line 113) | <K> void setKey(K key) {
    method getUserProperties (line 118) | @Override
    method getHeader (line 127) | @Override
    method getDefaultWindowScaner (line 135) | @Override
    method copy (line 141) | @Override
    method forward (line 157) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/running/WorkerThread.java
  class WorkerThread (line 64) | public class WorkerThread extends Thread {
    method WorkerThread (line 73) | public WorkerThread(String threadName,
    method run (line 105) | @Override
    method shutdown (line 121) | public void shutdown() {
    class PlanetaryEngine (line 126) | @SuppressWarnings("unchecked")
      method PlanetaryEngine (line 141) | public PlanetaryEngine(DefaultLitePullConsumer unionConsumer, Defaul...
      method start (line 172) | void start() throws Throwable {
      method runInLoop (line 180) | void runInLoop() throws Throwable {
      method doCommit (line 240) | void doCommit(HashSet<MessageQueue> set) throws Throwable {
      method maybeResetOffsetToFirst (line 254) | void maybeResetOffsetToFirst() throws Exception {
      method prepareTime (line 302) | long prepareTime(MessageExt messageExt, SourceSupplier.SourceProcess...
      method createShuffleTopic (line 318) | void createShuffleTopic() throws Throwable {
      method stop (line 334) | public synchronized void stop() {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/serialization/KeyValueDeserializer.java
  type KeyValueDeserializer (line 22) | public interface KeyValueDeserializer<K, V> {
    method configure (line 23) | default void configure(Object... args) throws Throwable {
    method deserialize (line 26) | Pair<K, V> deserialize(byte[] source) throws Throwable;

FILE: core/src/main/java/org/apache/rocketmq/streams/core/serialization/KeyValueSerializer.java
  type KeyValueSerializer (line 19) | public interface KeyValueSerializer<KEY, T> {
    method serialize (line 20) | byte[] serialize(KEY key, T data) throws Throwable;

FILE: core/src/main/java/org/apache/rocketmq/streams/core/serialization/ShuffleProtocol.java
  class ShuffleProtocol (line 33) | public class ShuffleProtocol {
    method merge (line 35) | public byte[] merge(byte[] keyBytes, byte[] valueBytes) {
    method split (line 56) | public Pair<byte[], byte[]> split(byte[] total) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/serialization/deImpl/KVJsonDeserializer.java
  class KVJsonDeserializer (line 28) | public class KVJsonDeserializer<K, V> extends ShuffleProtocol implements...
    method KVJsonDeserializer (line 33) | public KVJsonDeserializer() {
    method configure (line 43) | @Override
    method deserialize (line 57) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/serialization/serImpl/KVJsonSerializer.java
  class KVJsonSerializer (line 26) | public class KVJsonSerializer<K, V> extends ShuffleProtocol implements K...
    method KVJsonSerializer (line 29) | public KVJsonSerializer() {
    method serialize (line 38) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/state/AbstractStore.java
  class AbstractStore (line 32) | public abstract class AbstractStore {
    method putInRecover (line 36) | protected void putInRecover(String stateTopicQueueKey, byte[] key) {
    method putInCalculating (line 40) | protected void putInCalculating(String stateTopicQueueKey, byte[] key) {
    method getInCalculating (line 44) | protected Set<byte[]> getInCalculating(String stateTopicQueue) {
    method removeCalculating (line 48) | protected void removeCalculating(String stateTopicQueue) {
    method getAll (line 52) | protected Set<byte[]> getAll(String stateTopicQueue) {
    method whichStateTopicQueueBelongTo (line 57) | protected String whichStateTopicQueueBelongTo(byte[] key) {
    method removeAllKey (line 61) | protected void removeAllKey(byte[] key) {
    method removeAll (line 66) | protected void removeAll(String stateTopicQueue) {
    method convertSourceTopicQueue2StateTopicQueue (line 70) | protected MessageQueue convertSourceTopicQueue2StateTopicQueue(Message...
    method convertSourceTopicQueue2StateTopicQueue (line 80) | protected Set<MessageQueue> convertSourceTopicQueue2StateTopicQueue(Se...
    method stateTopic2SourceTopic (line 98) | protected static String stateTopic2SourceTopic(String stateTopic) {
    method buildKey (line 107) | protected String buildKey(MessageExt messageExt) {
    method buildKey (line 111) | protected String buildKey(MessageQueue messageQueue) {
    class Wrapper (line 115) | static class Wrapper {
      method putInRecover (line 121) | public void putInRecover(String stateTopicQueueKey, byte[] key) {
      method putInCalculating (line 133) | public void putInCalculating(String stateTopicQueueKey, byte[] key) {
      method getInCalculating (line 147) | public Set<byte[]> getInCalculating(String stateTopicQueue) {
      method getAll (line 151) | public Set<byte[]> getAll(String stateTopicQueue) {
      method whichStateTopicQueueBelongTo (line 163) | public String whichStateTopicQueueBelongTo(byte[] key) {
      method deleteByKey (line 184) | public void deleteByKey(byte[] key) {
      method removeCalculating (line 223) | public void removeCalculating(String stateTopicQueueKey) {
      method removeAll (line 227) | public void removeAll(String stateTopicQueueKey) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/state/RocketMQStore.java
  class RocketMQStore (line 60) | public class RocketMQStore extends AbstractStore implements StateStore {
    method RocketMQStore (line 72) | public RocketMQStore(DefaultMQProducer producer, RocksDBStore rocksDBS...
    method init (line 79) | @Override
    method recover (line 83) | @Override
    method waitIfNotReady (line 89) | @Override
    method get (line 111) | @Override
    method put (line 119) | @Override
    method searchStateLessThanWatermark (line 126) | @Override
    method searchByKeyPrefix (line 135) | @Override
    method delete (line 145) | @Override
    method persist (line 171) | @Override
    method loadState (line 214) | public void loadState(Set<MessageQueue> addQueues) throws Throwable {
    method removeState (line 252) | public void removeState(Set<MessageQueue> removeQueues) throws Throwab...
    method pullToLast (line 289) | private void pullToLast(DefaultLitePullConsumer consumer) throws Throw...
    method replayState (line 324) | private void replayState(List<MessageExt> msgs) throws Throwable {
    method createStateTopic (line 374) | private void createStateTopic(String stateTopic, boolean sourceTopicIs...
    method getTotalQueueNumAndClusters (line 389) | private Pair<Integer, Set<String>> getTotalQueueNumAndClusters(String ...
    method close (line 407) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/state/RocksDBStore.java
  class RocksDBStore (line 38) | public class RocksDBStore extends AbstractStore implements AutoCloseable {
    method RocksDBStore (line 47) | public RocksDBStore(String path) {
    method createRocksDB (line 51) | private void createRocksDB(String path) {
    method get (line 81) | public byte[] get(String columnFamilyName, byte[] key) throws RocksDBE...
    method put (line 89) | public void put(String columnFamilyName, byte[] key, byte[] value) thr...
    method searchStateLessThanWatermark (line 93) | public List<Pair<byte[], byte[]>> searchStateLessThanWatermark(String ...
    method searchByKeyPrefix (line 124) | public List<Pair<String, byte[]>> searchByKeyPrefix(String keyPrefix,
    method deleteByKey (line 155) | public void deleteByKey(String columnFamilyName, byte[] key) throws Ro...
    method close (line 159) | public void close() throws Exception {
    method skipWatermarkKey (line 168) | private boolean skipWatermarkKey(byte[] target) {
    method main (line 183) | public static void main(String[] args) throws Throwable {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/state/StateStore.java
  type StateStore (line 28) | public interface StateStore extends AutoCloseable {
    method init (line 29) | void init() throws Throwable;
    method recover (line 33) | void recover(Set<MessageQueue> addQueues, Set<MessageQueue> removeQueu...
    method waitIfNotReady (line 37) | void waitIfNotReady(MessageQueue messageQueue) throws RecoverStateStor...
    method get (line 40) | byte[] get(String columnFamily, byte[] key) throws Throwable;
    method put (line 42) | void put(MessageQueue stateTopicMessageQueue, String columnFamily, byt...
    method searchStateLessThanWatermark (line 44) | List<Pair<byte[], byte[]>> searchStateLessThanWatermark(String operato...
    method searchByKeyPrefix (line 46) | List<Pair<String, byte[]>> searchByKeyPrefix(String keyPrefix, ValueMa...
    method delete (line 48) | void delete(byte[] key) throws Throwable;
    method persist (line 50) | void persist(Set<MessageQueue> messageQueue) throws Throwable;

FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/TopologyBuilder.java
  class TopologyBuilder (line 33) | public class TopologyBuilder {
    method TopologyBuilder (line 44) | public TopologyBuilder(String jobId) {
    method addRealSource (line 48) | public <T> void addRealSource(String name, String topicName, Supplier<...
    method addRealNode (line 60) | public <T> void addRealNode(String name, String parentName, Supplier<?...
    method addRealSink (line 70) | public <T> void addRealSink(String name, String parentName, String top...
    method grouping (line 78) | private void grouping(String name, String parentName) {
    method getSourceTopic (line 92) | public Set<String> getSourceTopic() {
    method build (line 98) | @SuppressWarnings("unchecked")
    method getJobId (line 119) | public String getJobId() {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/real/ProcessorFactory.java
  class ProcessorFactory (line 26) | public class ProcessorFactory<T> implements RealProcessorFactory<T> {
    method ProcessorFactory (line 30) | public ProcessorFactory(String name, Supplier<? extends Processor<T>> ...
    method getName (line 35) | @Override
    method build (line 40) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/real/RealProcessorFactory.java
  type RealProcessorFactory (line 23) | public interface RealProcessorFactory<T> {
    method getName (line 24) | String getName();
    method build (line 26) | Processor<T> build();

FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/real/SinkFactory.java
  class SinkFactory (line 25) | public class SinkFactory<T> implements RealProcessorFactory<T> {
    method SinkFactory (line 30) | public SinkFactory(String name, Supplier<? extends Processor<T>> suppl...
    method getName (line 36) | @Override
    method build (line 41) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/real/SourceFactory.java
  class SourceFactory (line 26) | public class SourceFactory<T> implements RealProcessorFactory<T> {
    method SourceFactory (line 31) | public SourceFactory(String name, String topic, Supplier<Processor<T>>...
    method getName (line 37) | @Override
    method build (line 42) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/AbstractGraphNode.java
  class AbstractGraphNode (line 27) | public abstract class AbstractGraphNode implements GraphNode {
    method AbstractGraphNode (line 36) | public AbstractGraphNode(String name) {
    method getName (line 42) | @Override
    method addParent (line 47) | @Override
    method addChild (line 56) | @Override
    method getAllChild (line 66) | @Override
    method getAllParent (line 71) | @Override
    method shuffleNode (line 76) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/GraphNode.java
  type GraphNode (line 23) | public interface GraphNode {
    method getName (line 24) | String getName();
    method addParent (line 26) | void addParent(GraphNode graphNode);
    method addChild (line 29) | void addChild(GraphNode graphNode);
    method getAllChild (line 31) | List<GraphNode> getAllChild();
    method getAllParent (line 33) | List<GraphNode> getAllParent();
    method shuffleNode (line 35) | boolean shuffleNode();
    method addRealNode (line 38) | void addRealNode(TopologyBuilder builder);

FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/ProcessorNode.java
  class ProcessorNode (line 26) | public class ProcessorNode<T> extends AbstractGraphNode {
    method ProcessorNode (line 32) | public ProcessorNode(String name, String parentName, Supplier<Processo...
    method ProcessorNode (line 39) | public ProcessorNode(String name, List<String> parentNames, Supplier<P...
    method ProcessorNode (line 45) | public ProcessorNode(String name, List<String> parentNames, boolean sh...
    method ProcessorNode (line 52) | public ProcessorNode(String name, String parentName, boolean shuffle, ...
    method shuffleNode (line 60) | @Override
    method addRealNode (line 66) | @Override
    method equals (line 75) | @Override
    method toString (line 83) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/ShuffleProcessorNode.java
  class ShuffleProcessorNode (line 32) | public class ShuffleProcessorNode<T> extends ProcessorNode<T> {
    method ShuffleProcessorNode (line 35) | public ShuffleProcessorNode(String name, String parentName, Supplier<P...
    method addRealNode (line 39) | @Override
    method toString (line 54) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/SinkGraphNode.java
  class SinkGraphNode (line 24) | public class SinkGraphNode<T> extends AbstractGraphNode {
    method SinkGraphNode (line 29) | public SinkGraphNode(String name, String parentName, String topicName,...
    method addRealNode (line 37) | @Override
    method toString (line 42) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/SourceGraphNode.java
  class SourceGraphNode (line 26) | public class SourceGraphNode<T> extends AbstractGraphNode {
    method SourceGraphNode (line 31) | public SourceGraphNode(String name, String topicName, KeyValueDeserial...
    method addRealNode (line 37) | @Override
    method toString (line 42) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/util/ColumnFamilyUtil.java
  class ColumnFamilyUtil (line 11) | public class ColumnFamilyUtil {
    type ColumnFamilyEnum (line 19) | private enum ColumnFamilyEnum {
      method ColumnFamilyEnum (line 34) | ColumnFamilyEnum(String name, Predicate<String> keyCheckFunc) {
    method createColumnFamilies (line 43) | public static void createColumnFamilies(RocksDB rocksDB, ColumnFamilyO...
    method getColumnFamilyByKey (line 54) | public static String getColumnFamilyByKey(byte[] key) {
    method getColumnFamilyByKey (line 61) | public static String getColumnFamilyByKey(String key) {
    method getColumnFamilyHandleByName (line 70) | public static ColumnFamilyHandle getColumnFamilyHandleByName(String na...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/util/OperatorNameMaker.java
  class OperatorNameMaker (line 21) | public class OperatorNameMaker {
    method incrementAndGet (line 49) | private static int incrementAndGet() {
    method makeName (line 53) | public static String makeName(String prefix, String jobId) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/util/Pair.java
  class Pair (line 23) | public class Pair<T1, T2> {
    method Pair (line 27) | @JsonCreator
    method getKey (line 33) | public T1 getKey() {
    method setKey (line 37) | public void setKey(T1 key) {
    method getValue (line 41) | public T2 getValue() {
    method setValue (line 45) | public void setValue(T2 value) {
    method toString (line 49) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/util/RocketMQUtil.java
  class RocketMQUtil (line 46) | public class RocketMQUtil {
    method createNormalTopic (line 52) | public static void createNormalTopic(DefaultMQAdminExt mqAdmin, String...
    method createNormalTopic (line 96) | public static void createNormalTopic(DefaultMQAdminExt mqAdmin, String...
    method createStaticCompactTopic (line 123) | public static void createStaticCompactTopic(DefaultMQAdminExt mqAdmin,...
    method createStaticTopic (line 146) | public static void createStaticTopic(DefaultMQAdminExt mqAdmin, String...
    method createStaticTopicWithCommand (line 162) | private static void createStaticTopicWithCommand(String topic, int tot...
    method update2CompactTopicWithCommand (line 191) | private static void update2CompactTopicWithCommand(String topic, int q...
    method getCluster (line 212) | public static Set<String> getCluster(DefaultMQAdminExt mqAdmin) throws...
    method check (line 217) | private static boolean check(DefaultMQAdminExt mqAdmin, String topicNa...
    method checkWhetherExist (line 239) | public static boolean checkWhetherExist(String topic) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/util/Utils.java
  class Utils (line 38) | public class Utils {
    method buildKey (line 51) | public static String buildKey(String brokerName, String topic, int que...
    method buildKey (line 55) | public static String buildKey(String key, String... args) {
    method split (line 76) | public static String[] split(String source) {
    method split (line 80) | public static String[] split(String source, String split) {
    method object2Byte (line 88) | public static byte[] object2Byte(Object target) throws JsonProcessingE...
    method byte2Object (line 97) | public static <B> B byte2Object(byte[] source, Class<B> clazz) throws ...
    method format (line 107) | public static String format(long timestamp) {
    method toHexString (line 112) | public static String toHexString(Object obj) {
    method long2Bytes (line 131) | public static byte[] long2Bytes(long time) {
    method bytes2Long (line 137) | public static long bytes2Long(byte[] bytes) {
    method watermarkKeyBytes (line 148) | public static byte[] watermarkKeyBytes(MessageQueue stateTopicMessageQ...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/JoinType.java
  type JoinType (line 19) | public enum JoinType {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/StreamType.java
  type StreamType (line 19) | public enum StreamType {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/Time.java
  class Time (line 23) | public class Time {
    method Time (line 28) | private Time(long size, TimeUnit unit) {
    method getUnit (line 33) | public TimeUnit getUnit() {
    method getSize (line 38) | public long getSize() {
    method toMilliseconds (line 43) | public long toMilliseconds() {
    method toMillSecond (line 47) | public long toMillSecond() {
    method of (line 51) | public static Time of(long size, TimeUnit unit) {
    method milliseconds (line 55) | public static Time milliseconds(long milliseconds) {
    method seconds (line 59) | public static Time seconds(long seconds) {
    method minutes (line 63) | public static Time minutes(long minutes) {
    method hours (line 67) | public static Time hours(long hours) {
    method days (line 71) | public static Time days(long days) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/TimeType.java
  type TimeType (line 19) | public enum TimeType {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/Window.java
  class Window (line 21) | public class Window {
    method Window (line 26) | public Window(long startTime, long endTime) {
    method getStartTime (line 31) | public long getStartTime() {
    method setStartTime (line 35) | public void setStartTime(long startTime) {
    method getEndTime (line 39) | public long getEndTime() {
    method setEndTime (line 43) | public void setEndTime(long endTime) {
    method toString (line 48) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/WindowBuilder.java
  class WindowBuilder (line 20) | public class WindowBuilder {
    method tumblingWindow (line 22) | public static WindowInfo tumblingWindow(Time size) {
    method slidingWindow (line 32) | public static WindowInfo slidingWindow(Time size, Time slideStep) {
    method sessionWindow (line 42) | public static WindowInfo sessionWindow(Time size) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/WindowInfo.java
  class WindowInfo (line 20) | public class WindowInfo {
    method getWindowType (line 32) | public WindowType getWindowType() {
    method setWindowType (line 36) | public void setWindowType(WindowType windowType) {
    method getWindowSize (line 40) | public Time getWindowSize() {
    method setWindowSize (line 44) | public void setWindowSize(Time windowSize) {
    method getWindowSlide (line 48) | public Time getWindowSlide() {
    method setWindowSlide (line 52) | public void setWindowSlide(Time windowSlide) {
    method getSessionTimeout (line 56) | public Time getSessionTimeout() {
    method setSessionTimeout (line 60) | public void setSessionTimeout(Time sessionTimeout) {
    method getJoinStream (line 64) | public JoinStream getJoinStream() {
    method setJoinStream (line 68) | public void setJoinStream(JoinStream joinStream) {
    type WindowType (line 72) | public enum WindowType {
    class JoinStream (line 78) | public static class JoinStream {
      method JoinStream (line 82) | public JoinStream(JoinType joinType, StreamType streamType) {
      method getJoinType (line 87) | public JoinType getJoinType() {
      method setJoinType (line 91) | public void setJoinType(JoinType joinType) {
      method getStreamType (line 95) | public StreamType getStreamType() {
      method setStreamType (line 99) | public void setStreamType(StreamType streamType) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/WindowKey.java
  class WindowKey (line 25) | public class WindowKey {
    method WindowKey (line 36) | public WindowKey(String operatorName, String key2String, Long windowEn...
    method getOperatorName (line 43) | public String getOperatorName() {
    method setOperatorName (line 47) | public void setOperatorName(String operatorName) {
    method getWindowStart (line 51) | public Long getWindowStart() {
    method setWindowStart (line 55) | public void setWindowStart(Long windowStart) {
    method getWindowEnd (line 59) | public Long getWindowEnd() {
    method setWindowEnd (line 63) | public void setWindowEnd(Long windowEnd) {
    method getKey2String (line 67) | public String getKey2String() {
    method setKey2String (line 71) | public void setKey2String(String key2String) {
    method getKeyAndWindow (line 76) | public String getKeyAndWindow() {
    method byte2WindowKey (line 87) | public static WindowKey byte2WindowKey(byte[] source) {
    method windowKey2Byte (line 94) | public static byte[] windowKey2Byte(WindowKey windowKey) {
    method hashCode (line 102) | @Override
    method equals (line 120) | @Override
    method toString (line 131) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/WindowState.java
  class WindowState (line 37) | public class WindowState<K, V> implements Serializable {
    method WindowState (line 49) | public WindowState() {
    method WindowState (line 52) | public WindowState(K key, V value, long recordLastTimestamp) throws Js...
    method getKey (line 67) | public K getKey() {
    method setKey (line 71) | public void setKey(K key) throws JsonProcessingException {
    method getValue (line 79) | public V getValue() {
    method setValue (line 83) | public void setValue(V value) throws JsonProcessingException {
    method getKeyBytes (line 91) | public byte[] getKeyBytes() {
    method setKeyBytes (line 95) | public void setKeyBytes(byte[] keyBytes) {
    method getValueBytes (line 99) | public byte[] getValueBytes() {
    method setValueBytes (line 104) | public void setValueBytes(byte[] valueBytes) {
    method getValueClazz (line 108) | @SuppressWarnings("unchecked")
    method setValueClazz (line 113) | public void setValueClazz(Class<?> valueClazz) {
    method setKeyClazz (line 117) | public void setKeyClazz(Class<?> keyClazz) {
    method getKeyClazz (line 121) | @SuppressWarnings("unchecked")
    method getRecordEarliestTimestamp (line 126) | public long getRecordEarliestTimestamp() {
    method setRecordEarliestTimestamp (line 130) | public void setRecordEarliestTimestamp(long recordEarliestTimestamp) {
    method getRecordLastTimestamp (line 134) | public long getRecordLastTimestamp() {
    method setRecordLastTimestamp (line 138) | public void setRecordLastTimestamp(long recordLastTimestamp) {
    method windowState2Byte (line 143) | public static byte[] windowState2Byte(WindowState<?, ?> state) throws ...
    method byte2WindowState (line 201) | public static <K,V> WindowState<K,V> byte2WindowState(byte[] bytes) th...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/WindowStore.java
  class WindowStore (line 31) | public class WindowStore<K, V> {
    method WindowStore (line 39) | public WindowStore(StateStore stateStore,
    method put (line 47) | public void put(MessageQueue stateTopicMessageQueue, WindowKey windowK...
    method get (line 55) | public WindowState<K, V> get(WindowKey windowKey) throws Throwable {
    method searchLessThanWatermark (line 61) | public List<Pair<WindowKey, WindowState<K, V>>> searchLessThanWatermar...
    method searchMatchKeyPrefix (line 66) | public List<Pair<WindowKey, WindowState<K, V>>> searchMatchKeyPrefix(S...
    method deleteByKey (line 72) | public void deleteByKey(WindowKey windowKey) throws Throwable {
    method deserializerState (line 80) | private List<Pair<WindowKey, WindowState<K, V>>> deserializerState(Lis...
    method deserializerState (line 98) | private WindowState<K, V> deserializerState(byte[] source) throws Thro...

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AbstractWindowFire.java
  class AbstractWindowFire (line 30) | public abstract class AbstractWindowFire<K, V> implements WindowFire<K, ...
    method AbstractWindowFire (line 35) | public AbstractWindowFire(StreamContext<V> context,
    method commitWatermark (line 43) | void commitWatermark(long watermark) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AccumulatorSessionWindowFire.java
  class AccumulatorSessionWindowFire (line 40) | public class AccumulatorSessionWindowFire<K, R, V, OV> extends AbstractW...
    method AccumulatorSessionWindowFire (line 45) | public AccumulatorSessionWindowFire(WindowStore<K, Accumulator<R, OV>>...
    method fire (line 53) | public List<WindowKey> fire(String operatorName, long watermark) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AccumulatorWindowFire.java
  class AccumulatorWindowFire (line 38) | public class AccumulatorWindowFire<K, R, V, OV> extends AbstractWindowFi...
    method AccumulatorWindowFire (line 43) | public AccumulatorWindowFire(WindowStore<K, Accumulator<R, OV>> window...
    method fire (line 52) | public List<WindowKey> fire(String operatorName, long watermark) {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AggregateSessionWindowFire.java
  class AggregateSessionWindowFire (line 38) | public class AggregateSessionWindowFire<K, V, OV> extends AbstractWindow...
    method AggregateSessionWindowFire (line 43) | public AggregateSessionWindowFire(WindowStore<K, OV> windowStore,
    method fire (line 51) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AggregateWindowFire.java
  class AggregateWindowFire (line 37) | public class AggregateWindowFire<K, V, OV> extends AbstractWindowFire<K,...
    method AggregateWindowFire (line 42) | public AggregateWindowFire(WindowStore<K, OV> windowStore,
    method fire (line 50) | @Override

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/IdleWindowScaner.java
  class IdleWindowScaner (line 32) | public class IdleWindowScaner implements AutoCloseable {
    method IdleWindowScaner (line 48) | public IdleWindowScaner(Integer maxIdleTime, ScheduledExecutorService ...
    method putAccumulatorWindowCallback (line 60) | public void putAccumulatorWindowCallback(WindowKey windowKey, long wat...
    method putAccumulatorSessionWindowCallback (line 73) | public void putAccumulatorSessionWindowCallback(WindowKey windowKey, l...
    method putAggregateWindowCallback (line 86) | public void putAggregateWindowCallback(WindowKey windowKey, long water...
    method putAggregateSessionWindowCallback (line 99) | public void putAggregateSessionWindowCallback(WindowKey windowKey, lon...
    method putJoinWindowCallback (line 112) | public void putJoinWindowCallback(WindowKey windowKey, long watermark,...
    method removeOldAccumulatorSession (line 125) | public void removeOldAccumulatorSession(WindowKey oldWindowKey) {
    method removeOldAggregateSession (line 137) | public void removeOldAggregateSession(WindowKey oldWindowKey) {
    method removeWindowKey (line 146) | public void removeWindowKey(WindowKey windowKey) {
    method scanAndFireWindow (line 158) | private void scanAndFireWindow() throws Throwable {
    method doFire (line 203) | private void doFire(WindowKey windowKey, Type type, long watermark) th...
    method close (line 255) | @Override
    class TimeType (line 260) | static class TimeType {
      method TimeType (line 265) | public TimeType(Type type, long updateTime, long watermark) {
      method getType (line 271) | public Type getType() {
      method setType (line 275) | public void setType(Type type) {
      method getUpdateTime (line 279) | public long getUpdateTime() {
      method setUpdateTime (line 283) | public void setUpdateTime(long updateTime) {
      method getWatermark (line 287) | public long getWatermark() {
      method setWatermark (line 291) | public void setWatermark(long watermark) {
    type Type (line 296) | enum Type {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/JoinWindowFire.java
  class JoinWindowFire (line 45) | public class JoinWindowFire<K, V1, V2, OUT> {
    method JoinWindowFire (line 56) | public JoinWindowFire(JoinType joinType,
    method fire (line 72) | public List<WindowKey> fire(String operatorName, long watermark, Strea...
    method convert (line 208) | @SuppressWarnings("unchecked")
    method commitWatermark (line 213) | void commitWatermark(long watermark) throws Throwable {

FILE: core/src/main/java/org/apache/rocketmq/streams/core/window/fire/WindowFire.java
  type WindowFire (line 24) | public interface WindowFire<K, V> {
    method fire (line 26) | List<WindowKey> fire(String operatorName, long watermark);
    method convert (line 29) | @SuppressWarnings("unchecked")

FILE: core/src/test/java/org/apache/rocketmq/streams/core/Num.java
  class Num (line 19) | public class Num {
    method getNumber (line 22) | public Integer getNumber() {
    method setNumber (line 26) | public void setNumber(Integer number) {

FILE: core/src/test/java/org/apache/rocketmq/streams/core/Test.java
  class Test (line 19) | public class Test {

FILE: core/src/test/java/org/apache/rocketmq/streams/core/User.java
  class User (line 19) | public class User {
    method getName (line 22) | public String getName() {
    method setName (line 26) | public void setName(String name) {

FILE: core/src/test/java/org/apache/rocketmq/streams/core/function/AggregateSupplier.java
  class AggregateSupplier (line 29) | public class AggregateSupplier {
    method main (line 30) | public static void main(String[] args) throws Throwable {
    method object2Byte (line 46) | protected static byte[] object2Byte(Object obj) throws JsonProcessingE...
    method byte2Object (line 79) | @SuppressWarnings("unchecked")

FILE: core/src/test/java/org/apache/rocketmq/streams/core/runtime/WindowStateTests.java
  class WindowStateTests (line 23) | public class WindowStateTests {
    method main (line 24) | public static void main(String[] args) throws Throwable {

FILE: core/src/test/java/org/apache/rocketmq/streams/core/state/RocksDBStoreTest.java
  class RocksDBStoreTest (line 27) | public class RocksDBStoreTest {
    method main (line 28) | public static void main(String[] args) throws Throwable {
    method byte2WindowKey (line 63) | private static WindowKey byte2WindowKey(byte[] source) {
    method key2Byte (line 69) | private static byte[] key2Byte(WindowKey windowKey) {

FILE: core/src/test/java/org/apache/rocketmq/streams/core/util/UtilsTests.java
  class UtilsTests (line 24) | public class UtilsTests {
    method main (line 26) | public static void main(String[] args) throws JsonProcessingException {
    class User (line 37) | static class User {
      method getName (line 40) | public String getName() {
      method setName (line 44) | public void setName(String name) {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/WordCount.java
  class WordCount (line 31) | public class WordCount {
    method main (line 32) | public static void main(String[] args) throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/WordCountFromFirstOffset.java
  class WordCountFromFirstOffset (line 34) | public class WordCountFromFirstOffset {
    method main (line 35) | public static void main(String[] args) {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/joinWindow/JoinWindow.java
  class JoinWindow (line 42) | public class JoinWindow {
    method main (line 43) | public static void main(String[] args) throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/joinWindow/JoinWithoutWindow.java
  class JoinWithoutWindow (line 33) | public class JoinWithoutWindow {
    method main (line 34) | public static void main(String[] args) throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Demo.java
  class Demo (line 37) | public class Demo {
    method main (line 38) | public static void main(String[] args) throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Grade.java
  class Grade (line 19) | public class Grade {
    method Grade (line 26) | public Grade(int grade, int score) {
    method Grade (line 31) | public Grade() {
    method getGrade (line 35) | public int getGrade() {
    method setGrade (line 39) | public void setGrade(int grade) {
    method getScore (line 43) | public int getScore() {
    method setScore (line 47) | public void setScore(int score) {
    method getTimestamp (line 51) | public long getTimestamp() {
    method setTimestamp (line 55) | public void setTimestamp(long timestamp) {
    method toString (line 59) | @Override public String toString() {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Num.java
  class Num (line 19) | public class Num {
    method Num (line 23) | public Num(){
    method Num (line 26) | public Num(String name, Integer num) {
    method getName (line 31) | public String getName() {
    method setName (line 35) | public void setName(String name) {
    method getNum (line 39) | public Integer getNum() {
    method setNum (line 43) | public void setNum(Integer num) {
    method toString (line 47) | @Override

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Order.java
  class Order (line 3) | public class Order {
    method Order (line 8) | public Order() {
    method Order (line 12) | public Order(String type, Integer price, String customer) {
    method getType (line 18) | public String getType() {
    method getCustomer (line 22) | public String getCustomer() {
    method getPrice (line 26) | public Integer getPrice() {
    method setType (line 30) | public void setType(String type) {
    method setCustomer (line 34) | public void setCustomer(String customer) {
    method setPrice (line 38) | public void setPrice(Integer price) {
    method toString (line 42) | @Override

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Union.java
  class Union (line 19) | public class Union {
    method Union (line 24) | public Union(String name, Integer age, Integer num) {
    method Union (line 30) | public Union() {
    method getName (line 33) | public String getName() {
    method setName (line 37) | public void setName(String name) {
    method getAge (line 41) | public Integer getAge() {
    method setAge (line 45) | public void setAge(Integer age) {
    method getNum (line 49) | public Integer getNum() {
    method setNum (line 53) | public void setNum(Integer num) {
    method toString (line 57) | @Override

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/pojo/User.java
  class User (line 19) | public class User {
    method User (line 25) | public User() {
    method User (line 28) | public User(String name, Integer age) {
    method User (line 33) | public User(String name, Integer age, long timestamp) {
    method getName (line 39) | public String getName() {
    method setName (line 43) | public void setName(String name) {
    method getAge (line 47) | public Integer getAge() {
    method setAge (line 51) | public void setAge(Integer age) {
    method getTimestamp (line 55) | public long getTimestamp() {
    method setTimestamp (line 59) | public void setTimestamp(long timestamp) {
    method toString (line 63) | @Override

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/sink/WordCountSink.java
  class WordCountSink (line 41) | public class WordCountSink {
    method main (line 42) | public static void main(String[] args) throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/SessionWindowCount.java
  class SessionWindowCount (line 39) | public class SessionWindowCount {
    method main (line 42) | public static void main(String[] args) throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/SlideWindowCount.java
  class SlideWindowCount (line 37) | public class SlideWindowCount {
    method main (line 38) | public static void main(String[] args) throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowAvg.java
  class WindowAvg (line 39) | public class WindowAvg {
    method main (line 40) | public static void main(String[] args) throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowCount.java
  class WindowCount (line 41) | public class WindowCount {
    method main (line 42) | public static void main(String[] args) throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowMin.java
  class WindowMin (line 41) | public class WindowMin {
    method main (line 44) | public static void main(String[] args) throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowOrderCount.java
  class WindowOrderCount (line 17) | public class WindowOrderCount {
    method main (line 18) | public static void main(String[] args) throws Throwable {
    method getOrder1 (line 39) | private static StreamBuilder getOrder1() throws Throwable {
    method getOrder2 (line 59) | private static StreamBuilder getOrder2() throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowSum.java
  class WindowSum (line 41) | public class WindowSum {
    method main (line 44) | public static void main(String[] args) throws Throwable {

FILE: example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowWordCount.java
  class WindowWordCount (line 42) | public class WindowWordCount {
    method main (line 43) | public static void main(String[] args) throws Throwable {
Condensed preview — 132 files, each showing path, character count, and a content snippet. Download the .json file or copy for the full structured content (493K chars).
[
  {
    "path": ".asf.yaml",
    "chars": 355,
    "preview": "github:\n  features:\n    # Enable issue management\n    issues: true\n    # Enable wiki\n    wiki: true\n\n  protected_branche"
  },
  {
    "path": ".gitignore",
    "chars": 323,
    "preview": "target/\n.DS_Store\n!.mvn/wrapper/maven-wrapper.jar\n*.versionsBackup\n.gradle/\n\n### STS ###\n.apt_generated\n.classpath\n.fact"
  },
  {
    "path": ".travis.yml",
    "chars": 632,
    "preview": "dist: trusty\n \nlanguage: java\n\njdk:\n    - oraclejdk8\n    - oraclejdk9\n    - oraclejdk11\n    - openjdk8\n\nbefore_install:\n"
  },
  {
    "path": "LICENSE",
    "chars": 11356,
    "preview": "                                 Apache License\n                           Version 2.0, January 2004\n                   "
  },
  {
    "path": "NOTICE",
    "chars": 169,
    "preview": "Apache RocketMQ\nCopyright 2016-2023 The Apache Software Foundation\n\nThis product includes software developed at\nThe Apac"
  },
  {
    "path": "README.md",
    "chars": 5047,
    "preview": "# RocketMQ Streams \n[![Build Status](https://app.travis-ci.com/apache/rocketmq-streams.svg?branch=main)](https://app.tra"
  },
  {
    "path": "build_without_test.sh",
    "chars": 835,
    "preview": "#!/bin/sh\n\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements.  See t"
  },
  {
    "path": "core/pom.xml",
    "chars": 2093,
    "preview": "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n<project xmlns=\"http://maven.apache.org/POM/4.0.0\" xmlns:xsi=\"http://www.w3.org/2"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/RocketMQStream.java",
    "chars": 3516,
    "preview": "package org.apache.rocketmq.streams.core;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * co"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/common/Constant.java",
    "chars": 1960,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/exception/DataProcessThrowable.java",
    "chars": 1418,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/exception/DeserializeThrowable.java",
    "chars": 1490,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/exception/RStreamsException.java",
    "chars": 1479,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/exception/RecoverStateStoreThrowable.java",
    "chars": 1454,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/AggregateAction.java",
    "chars": 954,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/FilterAction.java",
    "chars": 1039,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/ForeachAction.java",
    "chars": 938,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/SelectAction.java",
    "chars": 945,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/ValueJoinAction.java",
    "chars": 955,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/ValueMapperAction.java",
    "chars": 946,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/Accumulator.java",
    "chars": 1272,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/AvgAccumulator.java",
    "chars": 2545,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/CountAccumulator.java",
    "chars": 1666,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/MinAccumulator.java",
    "chars": 1883,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AccumulatorSupplier.java",
    "chars": 4444,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AddTagSupplier.java",
    "chars": 2133,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AggregateSupplier.java",
    "chars": 4377,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/FilterSupplier.java",
    "chars": 2068,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/ForeachSupplier.java",
    "chars": 2024,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/JoinAggregateSupplier.java",
    "chars": 6887,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/JoinWindowAggregateSupplier.java",
    "chars": 7488,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/KeySelectSupplier.java",
    "chars": 2039,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/MultiValueChangeSupplier.java",
    "chars": 2857,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/PrintSupplier.java",
    "chars": 2492,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SinkSupplier.java",
    "chars": 4480,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SourceSupplier.java",
    "chars": 3375,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SumAggregate.java",
    "chars": 2092,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/TimestampSelectorSupplier.java",
    "chars": 2323,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/ValueChangeSupplier.java",
    "chars": 2536,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/WindowAccumulatorSupplier.java",
    "chars": 16087,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/WindowAggregateSupplier.java",
    "chars": 15774,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/metadata/Data.java",
    "chars": 2222,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/metadata/StreamConfig.java",
    "chars": 1706,
    "preview": "package org.apache.rocketmq.streams.core.metadata;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or m"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/GroupedStream.java",
    "chars": 2329,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/GroupedStreamImpl.java",
    "chars": 11660,
    "preview": "package org.apache.rocketmq.streams.core.rstream;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or mo"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/JoinedStream.java",
    "chars": 8616,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/Pipeline.java",
    "chars": 3589,
    "preview": "package org.apache.rocketmq.streams.core.rstream;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or mo"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/RStream.java",
    "chars": 1886,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/RStreamImpl.java",
    "chars": 6804,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/StreamBuilder.java",
    "chars": 2947,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/WindowStream.java",
    "chars": 2123,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/WindowStreamImpl.java",
    "chars": 10789,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/AbstractProcessor.java",
    "chars": 5864,
    "preview": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or mo"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/AbstractWindowProcessor.java",
    "chars": 3846,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/MessageQueueListenerWrapper.java",
    "chars": 4824,
    "preview": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or mo"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/Processor.java",
    "chars": 1136,
    "preview": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or mo"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/RocketMQClient.java",
    "chars": 3125,
    "preview": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or mo"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/StreamContext.java",
    "chars": 1669,
    "preview": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or mo"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/StreamContextImpl.java",
    "chars": 5414,
    "preview": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or mo"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/WorkerThread.java",
    "chars": 15701,
    "preview": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or mo"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/serialization/KeyValueDeserializer.java",
    "chars": 1089,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/serialization/KeyValueSerializer.java",
    "chars": 963,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/serialization/ShuffleProtocol.java",
    "chars": 2629,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/serialization/deImpl/KVJsonDeserializer.java",
    "chars": 3033,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/serialization/serImpl/KVJsonSerializer.java",
    "chars": 2671,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/state/AbstractStore.java",
    "chars": 8368,
    "preview": "package org.apache.rocketmq.streams.core.state;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/state/RocketMQStore.java",
    "chars": 16841,
    "preview": "package org.apache.rocketmq.streams.core.state;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/state/RocksDBStore.java",
    "chars": 8510,
    "preview": "package org.apache.rocketmq.streams.core.state;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/state/StateStore.java",
    "chars": 2298,
    "preview": "package org.apache.rocketmq.streams.core.state;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/TopologyBuilder.java",
    "chars": 4596,
    "preview": "package org.apache.rocketmq.streams.core.topology;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or m"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/real/ProcessorFactory.java",
    "chars": 1484,
    "preview": "package org.apache.rocketmq.streams.core.topology.real;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/real/RealProcessorFactory.java",
    "chars": 1037,
    "preview": "package org.apache.rocketmq.streams.core.topology.real;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/real/SinkFactory.java",
    "chars": 1552,
    "preview": "package org.apache.rocketmq.streams.core.topology.real;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/real/SourceFactory.java",
    "chars": 1560,
    "preview": "package org.apache.rocketmq.streams.core.topology.real;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/AbstractGraphNode.java",
    "chars": 2418,
    "preview": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under "
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/GraphNode.java",
    "chars": 1241,
    "preview": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under "
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/ProcessorNode.java",
    "chars": 3023,
    "preview": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under "
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/ShuffleProcessorNode.java",
    "chars": 2729,
    "preview": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under "
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/SinkGraphNode.java",
    "chars": 1710,
    "preview": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under "
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/SourceGraphNode.java",
    "chars": 1798,
    "preview": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under "
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/util/ColumnFamilyUtil.java",
    "chars": 2643,
    "preview": "package org.apache.rocketmq.streams.core.util;\n\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.rock"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/util/OperatorNameMaker.java",
    "chars": 2797,
    "preview": "package org.apache.rocketmq.streams.core.util;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/util/Pair.java",
    "chars": 1602,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/util/RocketMQUtil.java",
    "chars": 10641,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/util/Utils.java",
    "chars": 5674,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/JoinType.java",
    "chars": 907,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/StreamType.java",
    "chars": 913,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/Time.java",
    "chars": 2066,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/TimeType.java",
    "chars": 910,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/Window.java",
    "chars": 1637,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/WindowBuilder.java",
    "chars": 1671,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/WindowInfo.java",
    "chars": 2698,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/WindowKey.java",
    "chars": 4255,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/WindowState.java",
    "chars": 8252,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/WindowStore.java",
    "chars": 4437,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AbstractWindowFire.java",
    "chars": 1964,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AccumulatorSessionWindowFire.java",
    "chars": 4486,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AccumulatorWindowFire.java",
    "chars": 4235,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AggregateSessionWindowFire.java",
    "chars": 4230,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AggregateWindowFire.java",
    "chars": 3986,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/IdleWindowScaner.java",
    "chars": 12408,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/JoinWindowFire.java",
    "chars": 10868,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/WindowFire.java",
    "chars": 1301,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/main/resources/logback.xml",
    "chars": 1715,
    "preview": "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n<configuration>\n    <appender name=\"DefaultAppender_inner\"\n              class=\"c"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/Num.java",
    "chars": 1038,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/Test.java",
    "chars": 867,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/User.java",
    "chars": 1022,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/function/AggregateSupplier.java",
    "chars": 3526,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/runtime/WindowStateTests.java",
    "chars": 1519,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/state/RocksDBStoreTest.java",
    "chars": 3231,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/util/UtilsTests.java",
    "chars": 1741,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/pom.xml",
    "chars": 898,
    "preview": "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n<project xmlns=\"http://maven.apache.org/POM/4.0.0\" xmlns:xsi=\"http://www.w3.org/2"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/WordCount.java",
    "chars": 2522,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/WordCountFromFirstOffset.java",
    "chars": 2759,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/joinWindow/JoinWindow.java",
    "chars": 3937,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/joinWindow/JoinWithoutWindow.java",
    "chars": 3685,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Demo.java",
    "chars": 2541,
    "preview": "package org.apache.rocketmq.streams.examples.pojo;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or m"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Grade.java",
    "chars": 1698,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Num.java",
    "chars": 1477,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Order.java",
    "chars": 982,
    "preview": "package org.apache.rocketmq.streams.examples.pojo;\n\npublic class Order {\n    private String type;        // drink, food,"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Union.java",
    "chars": 1707,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/User.java",
    "chars": 1881,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/sink/WordCountSink.java",
    "chars": 3414,
    "preview": "package org.apache.rocketmq.streams.examples.sink;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or m"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/SessionWindowCount.java",
    "chars": 2868,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/SlideWindowCount.java",
    "chars": 2602,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowAvg.java",
    "chars": 2654,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowCount.java",
    "chars": 2820,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowMin.java",
    "chars": 2956,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowOrderCount.java",
    "chars": 3102,
    "preview": "package org.apache.rocketmq.streams.examples.window;\n\nimport com.alibaba.fastjson.JSON;\nimport org.apache.rocketmq.commo"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowSum.java",
    "chars": 2932,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowWordCount.java",
    "chars": 3008,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOT"
  },
  {
    "path": "pom.xml",
    "chars": 8861,
    "preview": "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n<!--\n  Licensed to the Apache Software Foundation (ASF) under one or more\n  contr"
  }
]

About this extraction

This page contains the full source code of the apache/rocketmq-streams GitHub repository, extracted and formatted as plain text for AI agents and large language models (LLMs). The extraction includes 132 files (452.2 KB), approximately 102.6k tokens, and a symbol index with 803 extracted functions, classes, methods, constants, and types. Use this with OpenClaw, Claude, ChatGPT, Cursor, Windsurf, or any other AI tool that accepts text input. You can copy the full output to your clipboard or download it as a .txt file.

Extracted by GitExtract — free GitHub repo to text converter for AI. Built by Nikandr Surkov.

Copied to clipboard!