[
  {
    "path": ".asf.yaml",
    "content": "github:\n  features:\n    # Enable issue management\n    issues: true\n    # Enable wiki\n    wiki: true\n\n  protected_branches:\n\nnotifications:\n  commits:      commits@rocketmq.apache.org\n  issues:       commits@rocketmq.apache.org\n  pullrequests: commits@rocketmq.apache.org\n  jobs:         commits@rocketmq.apache.org\n  discussions:  dev@rocketmq.apache.org\n"
  },
  {
    "path": ".gitignore",
    "content": "target/\n.DS_Store\n!.mvn/wrapper/maven-wrapper.jar\n*.versionsBackup\n.gradle/\n\n### STS ###\n.apt_generated\n.classpath\n.factorypath\n.project\n.settings\n.springBeans\n\n### IntelliJ IDEA ###\n.idea\n*.iws\n*.iml\n*.ipr\n*.zip\n\n### NetBeans ###\nnbproject/private/\nbuild/\nnbbuild/\ndist/\nnbdist/\n\npom.xml.releaseBackup\nrelease.properties\n\n"
  },
  {
    "path": ".travis.yml",
    "content": "dist: trusty\n \nlanguage: java\n\njdk:\n    - oraclejdk8\n    - oraclejdk9\n    - oraclejdk11\n    - openjdk8\n\nbefore_install:\n  - echo 'MAVEN_OPTS=\"$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal\"' >> ~/.mavenrc\n  - cat ~/.mavenrc\n  - if [ \"$TRAVIS_OS_NAME\" == \"osx\" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi\n  - if [ \"$TRAVIS_OS_NAME\" == \"linux\" ]; then jdk_switcher use \"$CUSTOM_JDK\"; fi\n\nscript:\n  - travis_retry mvn -B clean apache-rat:check\n  - travis_retry mvn -B clean install cobertura:cobertura\n\nafter_success:\n  - bash <(curl -s https://codecov.io/bash) || echo 'Codecov failed to upload'\n"
  },
  {
    "path": "LICENSE",
    "content": "                                 Apache License\n                           Version 2.0, January 2004\n                        http://www.apache.org/licenses/\n\n   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION\n\n   1. Definitions.\n\n      \"License\" shall mean the terms and conditions for use, reproduction,\n      and distribution as defined by Sections 1 through 9 of this document.\n\n      \"Licensor\" shall mean the copyright owner or entity authorized by\n      the copyright owner that is granting the License.\n\n      \"Legal Entity\" shall mean the union of the acting entity and all\n      other entities that control, are controlled by, or are under common\n      control with that entity. For the purposes of this definition,\n      \"control\" means (i) the power, direct or indirect, to cause the\n      direction or management of such entity, whether by contract or\n      otherwise, or (ii) ownership of fifty percent (50%) or more of the\n      outstanding shares, or (iii) beneficial ownership of such entity.\n\n      \"You\" (or \"Your\") shall mean an individual or Legal Entity\n      exercising permissions granted by this License.\n\n      \"Source\" form shall mean the preferred form for making modifications,\n      including but not limited to software source code, documentation\n      source, and configuration files.\n\n      \"Object\" form shall mean any form resulting from mechanical\n      transformation or translation of a Source form, including but\n      not limited to compiled object code, generated documentation,\n      and conversions to other media types.\n\n      \"Work\" shall mean the work of authorship, whether in Source or\n      Object form, made available under the License, as indicated by a\n      copyright notice that is included in or attached to the work\n      (an example is provided in the Appendix below).\n\n      \"Derivative Works\" shall mean any work, whether in Source or Object\n      form, that is based on (or derived from) the Work and for which the\n      editorial revisions, annotations, elaborations, or other modifications\n      represent, as a whole, an original work of authorship. For the purposes\n      of this License, Derivative Works shall not include works that remain\n      separable from, or merely link (or bind by name) to the interfaces of,\n      the Work and Derivative Works thereof.\n\n      \"Contribution\" shall mean any work of authorship, including\n      the original version of the Work and any modifications or additions\n      to that Work or Derivative Works thereof, that is intentionally\n      submitted to Licensor for inclusion in the Work by the copyright owner\n      or by an individual or Legal Entity authorized to submit on behalf of\n      the copyright owner. For the purposes of this definition, \"submitted\"\n      means any form of electronic, verbal, or written communication sent\n      to the Licensor or its representatives, including but not limited to\n      communication on electronic mailing lists, source code control systems,\n      and issue tracking systems that are managed by, or on behalf of, the\n      Licensor for the purpose of discussing and improving the Work, but\n      excluding communication that is conspicuously marked or otherwise\n      designated in writing by the copyright owner as \"Not a Contribution.\"\n\n      \"Contributor\" shall mean Licensor and any individual or Legal Entity\n      on behalf of whom a Contribution has been received by Licensor and\n      subsequently incorporated within the Work.\n\n   2. Grant of Copyright License. Subject to the terms and conditions of\n      this License, each Contributor hereby grants to You a perpetual,\n      worldwide, non-exclusive, no-charge, royalty-free, irrevocable\n      copyright license to reproduce, prepare Derivative Works of,\n      publicly display, publicly perform, sublicense, and distribute the\n      Work and such Derivative Works in Source or Object form.\n\n   3. Grant of Patent License. Subject to the terms and conditions of\n      this License, each Contributor hereby grants to You a perpetual,\n      worldwide, non-exclusive, no-charge, royalty-free, irrevocable\n      (except as stated in this section) patent license to make, have made,\n      use, offer to sell, sell, import, and otherwise transfer the Work,\n      where such license applies only to those patent claims licensable\n      by such Contributor that are necessarily infringed by their\n      Contribution(s) alone or by combination of their Contribution(s)\n      with the Work to which such Contribution(s) was submitted. If You\n      institute patent litigation against any entity (including a\n      cross-claim or counterclaim in a lawsuit) alleging that the Work\n      or a Contribution incorporated within the Work constitutes direct\n      or contributory patent infringement, then any patent licenses\n      granted to You under this License for that Work shall terminate\n      as of the date such litigation is filed.\n\n   4. Redistribution. You may reproduce and distribute copies of the\n      Work or Derivative Works thereof in any medium, with or without\n      modifications, and in Source or Object form, provided that You\n      meet the following conditions:\n\n      (a) You must give any other recipients of the Work or\n          Derivative Works a copy of this License; and\n\n      (b) You must cause any modified files to carry prominent notices\n          stating that You changed the files; and\n\n      (c) You must retain, in the Source form of any Derivative Works\n          that You distribute, all copyright, patent, trademark, and\n          attribution notices from the Source form of the Work,\n          excluding those notices that do not pertain to any part of\n          the Derivative Works; and\n\n      (d) If the Work includes a \"NOTICE\" text file as part of its\n          distribution, then any Derivative Works that You distribute must\n          include a readable copy of the attribution notices contained\n          within such NOTICE file, excluding those notices that do not\n          pertain to any part of the Derivative Works, in at least one\n          of the following places: within a NOTICE text file distributed\n          as part of the Derivative Works; within the Source form or\n          documentation, if provided along with the Derivative Works; or,\n          within a display generated by the Derivative Works, if and\n          wherever such third-party notices normally appear. The contents\n          of the NOTICE file are for informational purposes only and\n          do not modify the License. You may add Your own attribution\n          notices within Derivative Works that You distribute, alongside\n          or as an addendum to the NOTICE text from the Work, provided\n          that such additional attribution notices cannot be construed\n          as modifying the License.\n\n      You may add Your own copyright statement to Your modifications and\n      may provide additional or different license terms and conditions\n      for use, reproduction, or distribution of Your modifications, or\n      for any such Derivative Works as a whole, provided Your use,\n      reproduction, and distribution of the Work otherwise complies with\n      the conditions stated in this License.\n\n   5. Submission of Contributions. Unless You explicitly state otherwise,\n      any Contribution intentionally submitted for inclusion in the Work\n      by You to the Licensor shall be under the terms and conditions of\n      this License, without any additional terms or conditions.\n      Notwithstanding the above, nothing herein shall supersede or modify\n      the terms of any separate license agreement you may have executed\n      with Licensor regarding such Contributions.\n\n   6. Trademarks. This License does not grant permission to use the trade\n      names, trademarks, service marks, or product names of the Licensor,\n      except as required for reasonable and customary use in describing the\n      origin of the Work and reproducing the content of the NOTICE file.\n\n   7. Disclaimer of Warranty. Unless required by applicable law or\n      agreed to in writing, Licensor provides the Work (and each\n      Contributor provides its Contributions) on an \"AS IS\" BASIS,\n      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or\n      implied, including, without limitation, any warranties or conditions\n      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A\n      PARTICULAR PURPOSE. You are solely responsible for determining the\n      appropriateness of using or redistributing the Work and assume any\n      risks associated with Your exercise of permissions under this License.\n\n   8. Limitation of Liability. In no event and under no legal theory,\n      whether in tort (including negligence), contract, or otherwise,\n      unless required by applicable law (such as deliberate and grossly\n      negligent acts) or agreed to in writing, shall any Contributor be\n      liable to You for damages, including any direct, indirect, special,\n      incidental, or consequential damages of any character arising as a\n      result of this License or out of the use or inability to use the\n      Work (including but not limited to damages for loss of goodwill,\n      work stoppage, computer failure or malfunction, or any and all\n      other commercial damages or losses), even if such Contributor\n      has been advised of the possibility of such damages.\n\n   9. Accepting Warranty or Additional Liability. While redistributing\n      the Work or Derivative Works thereof, You may choose to offer,\n      and charge a fee for, acceptance of support, warranty, indemnity,\n      or other liability obligations and/or rights consistent with this\n      License. However, in accepting such obligations, You may act only\n      on Your own behalf and on Your sole responsibility, not on behalf\n      of any other Contributor, and only if You agree to indemnify,\n      defend, and hold each Contributor harmless for any liability\n      incurred by, or claims asserted against, such Contributor by reason\n      of your accepting any such warranty or additional liability.\n\n   END OF TERMS AND CONDITIONS\n\n   APPENDIX: How to apply the Apache License to your work.\n\n      To apply the Apache License to your work, attach the following\n      boilerplate notice, with the fields enclosed by brackets \"[]\"\n      replaced with your own identifying information. (Don't include\n      the brackets!)  The text should be enclosed in the appropriate\n      comment syntax for the file format. We also recommend that a\n      file or class name and description of purpose be included on the\n      same \"printed page\" as the copyright notice for easier\n      identification within third-party archives.\n\n   Copyright [yyyy] [name of copyright owner]\n\n   Licensed under the Apache License, Version 2.0 (the \"License\");\n   you may not use this file except in compliance with the License.\n   You may obtain a copy of the License at\n\n       http://www.apache.org/licenses/LICENSE-2.0\n\n   Unless required by applicable law or agreed to in writing, software\n   distributed under the License is distributed on an \"AS IS\" BASIS,\n   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n   See the License for the specific language governing permissions and\n   limitations under the License."
  },
  {
    "path": "NOTICE",
    "content": "Apache RocketMQ\nCopyright 2016-2023 The Apache Software Foundation\n\nThis product includes software developed at\nThe Apache Software Foundation (http://www.apache.org/).\n"
  },
  {
    "path": "README.md",
    "content": "# RocketMQ Streams \n[![Build Status](https://app.travis-ci.com/apache/rocketmq-streams.svg?branch=main)](https://app.travis-ci.com/apache/rocketmq-streams)\n[![CodeCov](https://codecov.io/gh/apache/rocketmq-stream/branch/main/graph/badge.svg)](https://app.codecov.io/gh/apache/rocketmq-streams) \n[![GitHub release](https://img.shields.io/badge/release-download-orange.svg)](https://github.com/apache/rocketmq-streams/releases)\n[![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html)\n[![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\")\n[![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\")\n[![Twitter Follow](https://img.shields.io/twitter/follow/ApacheRocketMQ?style=social)](https://twitter.com/intent/follow?screen_name=ApacheRocketMQ)\n\n\n**RocketMQ Streams is a lightweight stream processing framework, application gains the stream processing ability by depending on RocketMQ Streams as an SDK.**\n\nIt offers a variety of features:\n\n* Function:\n  * One-to-one transform function, such as: filter, map, foreach\n  * Aggregate function, such as: sum, min, max, count, aggregate\n  * Generating function, such as: flatMap\n* Group by aggregate and window aggregate\n* Join stream\n* Custom serialization \n----------\n\n## Quick Start\nThis paragraph guides you running a stream processing with RocketMQ Streams.\n\n### Run RocketMQ 5.0 locally \n[RocketMQ quick-start](https://rocketmq.apache.org/docs/quick-start/)\n\nRocketMQ runs on all major operating systems and requires only a Java JDK version 8 or higher to be installed.\nTo check, run `java -version`:\n```shell\n$ java -version\njava version \"1.8.0_121\"\n```\n**1) Download RocketMQ**\n```shell\nwget https://archive.apache.org/dist/rocketmq/5.0.0/rocketmq-all-5.0.0-bin-release.zip\n\n# Unpack the release\n$ unzip rocketmq-all-5.0.0-bin-release.zip\n\n$ cd rocketmq-all-5.0.0-bin-release/bin\n```\n\n**2) Start NameServer**\n\nNameServer 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.\n\n```shell\n### start Name Server\n$ nohup sh mqnamesrv &\n\n### check whether Name Server is successfully started\n$ tail -f ~/logs/rocketmqlogs/namesrv.log\nThe Name Server boot success...\n```\n\n**2) Start Broker**\n\n```shell\n### start Broker\n$ nohup sh bin/mqbroker -n localhost:9876 &\n\n### check whether Broker is successfully started, eg: Broker's IP is 192.168.1.2, Broker's name is broker-a\n$ tail -f ~/logs/rocketmqlogs/broker.log\nThe broker[broker-a, 192.169.1.2:10911] boot success...\n```\n\n### Build stream processing application\n\n**1) Build application in IDE**\n\n**2) Add RocketMQ Streams dependency**\n```xml\n    <dependency>\n        <groupId>org.apache.rocketmq</groupId>\n        <artifactId>rocketmq-streams</artifactId>\n        <version>{current.version}</version>\n    </dependency>\n```\n\n**3) Build stream processing application**\n\n* create topic in RocketMQ before start the stream processing.\n```shell\nsh bin/mqadmin updateTopic -c ${clusterName} -t ${topicName} -r 8 -w 8 -n 127.0.0.1:9876\n```\n    \n    NOTE: the default clusterName is DefaultCluster in this quick-start doc, changes it with your RocketMQ cluster.\n\n* add your stream processing code, The following is an example. more examples are [here](./example/src/main/java/org/apache/rocketmq/streams/examples).\n```java\npublic static void main(String[] args) {\n        StreamBuilder builder = new StreamBuilder(\"wordCount\");\n\n        builder.source(\"sourceTopic\",  total -> {\n                    String value = new String(total, StandardCharsets.UTF_8);\n                    return new Pair<>(null, value);\n                })\n                .flatMap((ValueMapperAction<String, List<String>>) value -> {\n                    String[] splits = value.toLowerCase().split(\"\\\\W+\");\n                    return Arrays.asList(splits);\n                })\n                .keyBy(value -> value)\n                .count()\n                .toRStream()\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.put(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n\n        final CountDownLatch latch = new CountDownLatch(1);\n\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n                latch.countDown();\n            }\n        });\n\n        try {\n            rocketMQStream.start();\n            latch.await();\n        } catch (final Throwable e) {\n            System.exit(1);\n        }\n        System.exit(0);\n    }\n```\n"
  },
  {
    "path": "build_without_test.sh",
    "content": "#!/bin/sh\n\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements.  See the NOTICE file distributed with\n# this work for additional information regarding copyright ownership.\n# The ASF licenses this file to You under the Apache License, Version 2.0\n# (the \"License\"); you may not use this file except in compliance with\n# the License.  You may obtain a copy of the License ato\n#\n#     http://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing, software\n# distributed under the License is distributed on an \"AS IS\" BASIS,\n# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n# See the License for the specific language governing permissions and\n# limitations under the License.\n\nmvn clean package -Dmaven.test.skip=true\n"
  },
  {
    "path": "core/pom.xml",
    "content": "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n<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\">\n    <parent>\n        <artifactId>rocketmq-streams-all</artifactId>\n        <groupId>org.apache.rocketmq</groupId>\n        <version>1.1.2-SNAPSHOT</version>\n    </parent>\n    <modelVersion>4.0.0</modelVersion>\n\n    <artifactId>rocketmq-streams</artifactId>\n    <packaging>jar</packaging>\n\n    <properties>\n        <maven.compiler.source>8</maven.compiler.source>\n        <maven.compiler.target>8</maven.compiler.target>\n    </properties>\n\n    <dependencies>\n        <dependency>\n            <groupId>org.apache.rocketmq</groupId>\n            <artifactId>rocketmq-common</artifactId>\n            <version>${rocketmq.version}</version>\n        </dependency>\n        <dependency>\n            <groupId>org.apache.rocketmq</groupId>\n            <artifactId>rocketmq-client</artifactId>\n            <version>${rocketmq.version}</version>\n        </dependency>\n\n        <dependency>\n            <groupId>org.apache.rocketmq</groupId>\n            <artifactId>rocketmq-tools</artifactId>\n            <exclusions>\n                <exclusion>\n                    <groupId>com.google.protobuf</groupId>\n                    <artifactId>protobuf-java</artifactId>\n                </exclusion>\n            </exclusions>\n            <version>${rocketmq.version}</version>\n        </dependency>\n\n\n        <dependency>\n            <groupId>org.rocksdb</groupId>\n            <artifactId>rocksdbjni</artifactId>\n            <version>7.6.0</version>\n        </dependency>\n        <dependency>\n            <groupId>commons-io</groupId>\n            <artifactId>commons-io</artifactId>\n            <version>2.11.0</version>\n        </dependency>\n\n        <dependency>\n            <groupId>com.fasterxml.jackson.core</groupId>\n            <artifactId>jackson-databind</artifactId>\n            <version>2.13.4.2</version>\n        </dependency>\n\n    </dependencies>\n\n</project>"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/RocketMQStream.java",
    "content": "package org.apache.rocketmq.streams.core;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RStreamsException;\nimport org.apache.rocketmq.streams.core.metadata.StreamConfig;\nimport org.apache.rocketmq.streams.core.running.WorkerThread;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.List;\nimport java.util.Properties;\nimport java.util.concurrent.Executors;\nimport java.util.concurrent.ScheduledExecutorService;\nimport java.util.concurrent.atomic.AtomicBoolean;\nimport java.util.concurrent.atomic.AtomicInteger;\n\npublic class RocketMQStream {\n    private static final Logger logger = LoggerFactory.getLogger(RocketMQStream.class.getName());\n    private static final AtomicInteger index = new AtomicInteger(1);\n\n    private final TopologyBuilder topologyBuilder;\n    private final Properties properties;\n    private final List<WorkerThread> workerThreads = new ArrayList<>();\n    private final AtomicBoolean started = new AtomicBoolean(false);\n    private final ScheduledExecutorService executor = Executors.newScheduledThreadPool(\n                    StreamConfig.SCHEDULED_THREAD_NUM,\n                    r -> new Thread(r, \"ScanIdleWindowThread_\" + index.getAndIncrement()));\n\n    public RocketMQStream(TopologyBuilder topologyBuilder, Properties properties) {\n        this.topologyBuilder = topologyBuilder;\n        this.properties = properties;\n    }\n\n\n    public synchronized void start() {\n        String jobId = topologyBuilder.getJobId();\n        if (started.get()) {\n            logger.info(\"RocketMQStream has been started, jobId=[{}].\", jobId);\n            return;\n        }\n\n        this.started.compareAndSet(false, true);\n\n        //启动线程\n        try {\n            int threadNum = StreamConfig.STREAMS_PARALLEL_THREAD_NUM;\n            for (int i = 0; i < threadNum; i++) {\n                String threadName = String.join(\"_\", Constant.WORKER_THREAD_NAME, jobId, String.valueOf(i));\n\n                WorkerThread thread = new WorkerThread(threadName, topologyBuilder, this.properties, executor);\n\n                thread.start();\n                workerThreads.add(thread);\n            }\n        } catch (Throwable t) {\n            logger.error(\"start RocketMQStream error, jobId=[{}].\", jobId, t);\n            throw new RStreamsException(t);\n        }\n    }\n\n    public void stop() {\n        for (WorkerThread thread : workerThreads) {\n            thread.shutdown();\n        }\n        workerThreads.clear();\n        this.started.set(false);\n    }\n\n    public boolean isRunning() {\n        return this.started.get();\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/common/Constant.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n\npackage org.apache.rocketmq.streams.core.common;\n\npublic class Constant {\n\n    public static final String SHUFFLE_KEY_CLASS_NAME = \"shuffle.key.class.name\";\n\n    public static final String SHUFFLE_VALUE_CLASS_NAME = \"shuffle.value.class.name\";\n\n    public final static String STATE_TOPIC_SUFFIX = \"-stateTopic\";\n\n    public static final String SHUFFLE_TOPIC_SUFFIX = \"-shuffleTopic\";\n\n    public static final String SKIP_DATA_ERROR = \"skip_data_error\";\n\n    public static final String SPLIT = \"@\";\n\n    public static final String EMPTY_BODY = \"empty_body\";\n\n    public static final String TRUE = \"true\";\n\n    public static final String SOURCE_TIMESTAMP = \"source_timestamp\";\n\n    public static final String STREAM_TAG = \"stream_tag\";\n\n    public static final String WINDOW_START_TIME = \"window_start_time\";\n\n    public static final String WINDOW_END_TIME = \"window_end_time\";\n\n    public static final String WORKER_THREAD_NAME = \"ROCKETMQ_STREAMS\";\n\n    public static final String STATIC_TOPIC_BROKER_NAME = \"__syslo__global__\";\n\n    public static final String WATERMARK_KEY = \"watermark_key\";\n\n    public static final Long DEFAULT_CONSUME_OFFSET = 0L;\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/exception/DataProcessThrowable.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.exception;\n\npublic class DataProcessThrowable extends Throwable {\n    public DataProcessThrowable() {\n    }\n\n    public DataProcessThrowable(String message) {\n        super(message);\n    }\n\n    public DataProcessThrowable(String message, Throwable cause) {\n        super(message, cause);\n    }\n\n    public DataProcessThrowable(Throwable cause) {\n        super(cause);\n    }\n\n    public DataProcessThrowable(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {\n        super(message, cause, enableSuppression, writableStackTrace);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/exception/DeserializeThrowable.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.exception;\n\npublic class DeserializeThrowable extends Throwable {\n    private static final long serialVersionUID = 2154421351264920776L;\n\n    public DeserializeThrowable() {\n    }\n\n    public DeserializeThrowable(String message) {\n        super(message);\n    }\n\n    public DeserializeThrowable(String message, Throwable cause) {\n        super(message, cause);\n    }\n\n    public DeserializeThrowable(Throwable cause) {\n        super(cause);\n    }\n\n    public DeserializeThrowable(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {\n        super(message, cause, enableSuppression, writableStackTrace);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/exception/RStreamsException.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.exception;\n\npublic class RStreamsException extends RuntimeException {\n    private static final long serialVersionUID = 6729806497659471678L;\n\n    public RStreamsException() {\n    }\n\n    public RStreamsException(String message) {\n        super(message);\n    }\n\n    public RStreamsException(String message, Throwable cause) {\n        super(message, cause);\n    }\n\n    public RStreamsException(Throwable cause) {\n        super(cause);\n    }\n\n    public RStreamsException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {\n        super(message, cause, enableSuppression, writableStackTrace);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/exception/RecoverStateStoreThrowable.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.exception;\n\npublic class RecoverStateStoreThrowable extends Throwable {\n    public RecoverStateStoreThrowable() {\n    }\n\n    public RecoverStateStoreThrowable(String message) {\n        super(message);\n    }\n\n    public RecoverStateStoreThrowable(String message, Throwable cause) {\n        super(message, cause);\n    }\n\n    public RecoverStateStoreThrowable(Throwable cause) {\n        super(cause);\n    }\n\n    public RecoverStateStoreThrowable(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {\n        super(message, cause, enableSuppression, writableStackTrace);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/AggregateAction.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function;\n\npublic interface AggregateAction<K, V, OUT> {\n    OUT calculate(K key, V value, OUT accumulator);\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/FilterAction.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function;\n\n@FunctionalInterface\npublic interface FilterAction<T> {\n    /**\n     *\n     * @param value the target to filter\n     * @return pass if true;\n     */\n    boolean apply(final T value);\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/ForeachAction.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function;\n\n@FunctionalInterface\npublic interface ForeachAction<T> {\n    void apply(T value);\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/SelectAction.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function;\n\n\n@FunctionalInterface\npublic interface SelectAction<OUT, IN> {\n    OUT select(IN value);\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/ValueJoinAction.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function;\n\n\npublic interface ValueJoinAction<V1, V2, OUT> {\n\n    OUT apply(final V1 value1, final V2 value2);\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/ValueMapperAction.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function;\n\npublic interface ValueMapperAction<T, O> {\n    O convert(final T value) throws Throwable;\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/Accumulator.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.accumulator;\n\nimport java.util.Properties;\n\n//因为需要序列化/反序列化这个类，所以必须给field生成setter/getter方法\npublic interface Accumulator<V, R> {\n    void addValue(V value);\n\n    void merge(Accumulator<V, R> other);\n\n    /**\n     * invoked after the window fired.\n     *\n     * @param context the attached properties after window fired.\n     * @return the value.\n     */\n    R result(Properties context);\n\n    Accumulator<V, R> clone();\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/AvgAccumulator.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.accumulator;\n\nimport java.util.Properties;\n\npublic class AvgAccumulator<V> implements Accumulator<V, Double> {\n    private Double avg;\n    private Integer num;\n\n    @Override\n    public void addValue(V value) {\n        if (value instanceof Number) {\n            Number number = (Number) value;\n            Double valueToDouble = number.doubleValue();\n\n            if (avg == null) {\n                avg = valueToDouble;\n                num = 1;\n            } else {\n                avg = avg + (valueToDouble - avg) / (num + 1);\n                num++;\n            }\n        } else {\n            throw new IllegalArgumentException(\"Calculate avg, input is not a number. value=\" + value);\n        }\n    }\n\n    @Override\n    public void merge(Accumulator<V, Double> other) {\n        if (other instanceof AvgAccumulator) {\n            AvgAccumulator<V> otherAvgAccumulator = (AvgAccumulator) other;\n            Integer numOther = otherAvgAccumulator.getNum();\n            Double avgOther = otherAvgAccumulator.getAvg();\n\n            avg = avg + numOther / (num + numOther) * (avgOther - avg);\n            num = num + numOther;\n        } else {\n            throw new IllegalArgumentException(\"Merge avg, input is not a AvgAccumulator.\");\n        }\n    }\n\n    @Override\n    public Double result(Properties context) {\n        return avg;\n    }\n\n    @Override\n    public Accumulator<V, Double> clone() {\n        return new AvgAccumulator<>();\n    }\n\n    public Double getAvg() {\n        return avg;\n    }\n\n    public void setAvg(Double avg) {\n        this.avg = avg;\n    }\n\n    public Integer getNum() {\n        return num;\n    }\n\n    public void setNum(Integer num) {\n        this.num = num;\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/CountAccumulator.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.accumulator;\n\nimport java.util.Properties;\n\n//因为需要序列化/反序列化这个类，所以必须给field生成setter/getter方法\npublic class CountAccumulator<V> implements Accumulator<V, Integer> {\n    private Integer count = 0;\n\n    @Override\n    public void addValue(V value) {\n        count += 1;\n    }\n\n    @Override\n    public void merge(Accumulator<V, Integer> other) {\n        count += other.result(null);\n    }\n\n    @Override\n    public Integer result(Properties context) {\n        return count;\n    }\n\n    public Integer getCount() {\n        return count;\n    }\n\n    public void setCount(Integer count) {\n        this.count = count;\n    }\n\n    @Override\n    public Accumulator<V, Integer> clone() {\n        CountAccumulator<V> accumulator = new CountAccumulator<>();\n        accumulator.count = this.count;\n        return accumulator;\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/MinAccumulator.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.accumulator;\n\nimport java.util.Properties;\n\npublic class MinAccumulator<V> implements Accumulator<V, Double> {\n     private Number min;\n\n    @Override\n    public void addValue(V value) {\n        if (value instanceof Number) {\n            Number number = (Number) value;\n            if (min == null) {\n                min = number;\n            } else {\n                min = Math.min(min.doubleValue(), number.doubleValue());\n            }\n        } else {\n            throw new IllegalArgumentException(\"min but not a number. value=\" + value);\n        }\n\n    }\n\n    @Override\n    public void merge(Accumulator<V, Double> other) {\n        min = Math.min(min.doubleValue(), other.result(null));\n    }\n\n    @Override\n    public Double result(Properties context) {\n        return min.doubleValue();\n    }\n\n    @Override\n    public Accumulator<V, Double> clone() {\n        return new MinAccumulator<>();\n    }\n\n    public Number getMin() {\n        return min;\n    }\n\n    public void setMin(Number min) {\n        this.min = min;\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AccumulatorSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable;\nimport org.apache.rocketmq.streams.core.function.accumulator.Accumulator;\nimport org.apache.rocketmq.streams.core.function.SelectAction;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.state.StateStore;\nimport org.apache.rocketmq.streams.core.util.ColumnFamilyUtil;\n\nimport java.util.function.Supplier;\n\npublic class AccumulatorSupplier<K, V, R, OV> implements Supplier<Processor<V>> {\n    private final String currentName;\n    private final String parentName;\n    private SelectAction<R, V> selectAction;\n    private Accumulator<R, OV> accumulator;\n\n    public AccumulatorSupplier(String currentName, String parentName, SelectAction<R, V> selectAction, Accumulator<R, OV> accumulator) {\n        this.currentName = currentName;\n        this.parentName = parentName;\n        this.selectAction = selectAction;\n        this.accumulator = accumulator;\n    }\n\n    @Override\n    public Processor<V> get() {\n        return new AccumulatorProcessor(currentName, parentName, selectAction, accumulator);\n    }\n\n    private class AccumulatorProcessor extends AbstractProcessor<V> {\n        private final String currentName;\n        private final String parentName;\n        private StateStore stateStore;\n        private MessageQueue stateTopicMessageQueue;\n        private SelectAction<R, V> selectAction;\n        private Accumulator<R, OV> accumulator;\n\n        public AccumulatorProcessor(String currentName, String parentName, SelectAction<R, V> selectAction, Accumulator<R, OV> accumulator) {\n            this.currentName = currentName;\n            this.parentName = parentName;\n            this.selectAction = selectAction;\n            this.accumulator = accumulator;\n        }\n\n        @Override\n        public void preProcess(StreamContext<V> context) throws RecoverStateStoreThrowable {\n            super.preProcess(context);\n            this.stateStore = super.waitStateReplay();\n\n            String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX;\n            this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId());\n        }\n\n        @Override\n        public void process(V data) throws Throwable {\n            K key = this.context.getKey();\n            Accumulator<R, OV> value;\n\n            byte[] keyBytes = super.object2Byte(key);\n\n            byte[] valueBytes = stateStore.get(ColumnFamilyUtil.VALUE_STATE_CF, keyBytes);\n            if (valueBytes == null || valueBytes.length == 0) {\n                value = accumulator.clone();\n            } else {\n                value = super.byte2Object(valueBytes);\n            }\n\n            R select = selectAction.select(data);\n            value.addValue(select);\n\n            OV result = value.result(null);\n            byte[] newValueBytes = super.object2Byte(value);\n\n            stateStore.put(this.stateTopicMessageQueue, ColumnFamilyUtil.VALUE_STATE_CF, keyBytes, newValueBytes);\n\n            Data<K, OV> temp = new Data<>(key, result, this.context.getDataTime(), this.context.getHeader());\n            Data<K, V> convert = super.convert(temp);\n\n            this.context.forward(convert);\n        }\n\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AddTagSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\n\nimport java.util.Properties;\nimport java.util.function.Supplier;\n\npublic class AddTagSupplier<T> implements Supplier<Processor<T>> {\n    private final Properties properties = new Properties();\n\n    public AddTagSupplier(Supplier<Object> value) {\n\n        properties.put(Constant.STREAM_TAG, value.get());\n    }\n\n    public AddTagSupplier() {\n    }\n\n    @Override\n    public Processor<T> get() {\n        return new AddTagProcessor(properties);\n    }\n\n    class AddTagProcessor extends AbstractProcessor<T> {\n        private final Properties properties = new Properties();\n\n        public AddTagProcessor(Properties properties) {\n            this.properties.putAll(properties);\n        }\n\n        @Override\n        public void process(T data) throws Throwable {\n            Properties header = this.context.getHeader();\n            header.putAll(properties);\n            Data<Object, T> result = new Data<>(this.context.getKey(), data, this.context.getDataTime(), header);\n            this.context.forward(result);\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AggregateSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable;\nimport org.apache.rocketmq.streams.core.function.AggregateAction;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.state.StateStore;\nimport org.apache.rocketmq.streams.core.util.ColumnFamilyUtil;\n\nimport java.util.function.Supplier;\n\npublic class AggregateSupplier<K, V, OV> implements Supplier<Processor<V>> {\n    private final String currentName;\n    private final String parentName;\n    private Supplier<OV> initAction;\n    private AggregateAction<K, V, OV> aggregateAction;\n\n    public AggregateSupplier(String currentName, String parentName, Supplier<OV> initAction,\n                             AggregateAction<K, V, OV> aggregateAction) {\n        this.currentName = currentName;\n        this.parentName = parentName;\n        this.initAction = initAction;\n        this.aggregateAction = aggregateAction;\n    }\n\n    @Override\n    public Processor<V> get() {\n        return new AggregateProcessor(currentName, parentName, initAction, aggregateAction);\n    }\n\n    private class AggregateProcessor extends AbstractProcessor<V> {\n        private final String currentName;\n        private final String parentName;\n        private final Supplier<OV> initAction;\n        private final AggregateAction<K, V, OV> aggregateAction;\n        private StateStore stateStore;\n        private MessageQueue stateTopicMessageQueue;\n\n        public AggregateProcessor(String currentName, String parentName, Supplier<OV> initAction,\n                                  AggregateAction<K, V, OV> aggregateAction) {\n            this.currentName = currentName;\n            this.parentName = parentName;\n            this.initAction = initAction;\n            this.aggregateAction = aggregateAction;\n        }\n\n        @Override\n        public void preProcess(StreamContext<V> context) throws RecoverStateStoreThrowable {\n            super.preProcess(context);\n            this.stateStore = super.waitStateReplay();\n\n            String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX;\n            this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId());\n        }\n\n        @Override\n        public void process(V data) throws Throwable {\n            K key = this.context.getKey();\n            OV value;\n\n            byte[] keyBytes = super.object2Byte(key);\n\n            byte[] valueBytes = stateStore.get(ColumnFamilyUtil.VALUE_STATE_CF, keyBytes);\n            if (valueBytes == null || valueBytes.length == 0) {\n                value = initAction.get();\n            } else {\n                value = super.byte2Object(valueBytes);\n            }\n\n            OV result = aggregateAction.calculate(key, data, value);\n            byte[] newValueBytes = super.object2Byte(result);\n\n            stateStore.put(this.stateTopicMessageQueue, ColumnFamilyUtil.VALUE_STATE_CF, keyBytes, newValueBytes);\n\n            Data<K, OV> temp = new Data<>(key, result, this.context.getDataTime(), this.context.getHeader());\n            Data<K, V> convert = super.convert(temp);\n\n            this.context.forward(convert);\n        }\n\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/FilterSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.streams.core.function.FilterAction;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\n\nimport java.util.function.Supplier;\n\npublic class FilterSupplier<T> implements Supplier<Processor<T>> {\n    private FilterAction<T> filterAction;\n\n    public FilterSupplier(FilterAction<T> filterAction) {\n        this.filterAction = filterAction;\n    }\n\n    @Override\n    public Processor<T> get() {\n        return new FilterProcessor(filterAction);\n    }\n\n    private class FilterProcessor extends AbstractProcessor<T> {\n        private final FilterAction<T> filterAction;\n\n        public FilterProcessor(FilterAction<T> filterAction) {\n            this.filterAction = filterAction;\n        }\n\n        @Override\n        public  void process(T data) throws Throwable {\n            boolean pass = filterAction.apply(data);\n            if (pass) {\n                Data<Object, T> result = new Data<>(this.context.getKey(), data, this.context.getDataTime(), this.context.getHeader());\n                this.context.forward(result);\n            }\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/ForeachSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.streams.core.function.ForeachAction;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\n\nimport java.util.function.Supplier;\n\npublic class ForeachSupplier<T> implements Supplier<Processor<T>> {\n    private ForeachAction<T> foreachAction;\n\n    public ForeachSupplier(ForeachAction<T> foreachAction) {\n        this.foreachAction = foreachAction;\n    }\n\n    @Override\n    public Processor<T> get() {\n        return new ForeachProcessor(this.foreachAction);\n    }\n\n    class ForeachProcessor extends AbstractProcessor<T> {\n        private ForeachAction<T> foreachAction;\n\n        public ForeachProcessor(ForeachAction<T> foreachAction) {\n            this.foreachAction = foreachAction;\n        }\n\n        @Override\n        public void process(T data) throws Throwable {\n            this.foreachAction.apply(data);\n            Data<Object, T> result = new Data<>(this.context.getKey(), data, this.context.getDataTime(), this.context.getHeader());\n            this.context.forward(result);\n        }\n    }\n\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/JoinAggregateSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable;\nimport org.apache.rocketmq.streams.core.function.ValueJoinAction;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.util.ColumnFamilyUtil;\nimport org.apache.rocketmq.streams.core.window.JoinType;\nimport org.apache.rocketmq.streams.core.window.StreamType;\nimport org.apache.rocketmq.streams.core.state.StateStore;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.Properties;\nimport java.util.function.Supplier;\n\npublic class JoinAggregateSupplier<K, V1, V2, OUT> implements Supplier<Processor<? super OUT>> {\n    private static final Logger logger = LoggerFactory.getLogger(JoinAggregateSupplier.class.getName());\n\n    private String name;\n    private JoinType joinType;\n    private final ValueJoinAction<V1, V2, OUT> joinAction;\n\n    public JoinAggregateSupplier(String name, JoinType joinType, ValueJoinAction<V1, V2, OUT> joinAction) {\n        this.name = name;\n        this.joinType = joinType;\n        this.joinAction = joinAction;\n    }\n\n    @Override\n    public Processor<Object> get() {\n        return new JoinStreamAggregateProcessor(name, joinType, joinAction);\n    }\n\n    private class JoinStreamAggregateProcessor extends AbstractProcessor<Object> {\n        private String name;\n        private JoinType joinType;\n        private final ValueJoinAction<V1, V2, OUT> joinAction;\n        private MessageQueue stateTopicMessageQueue;\n        private StateStore stateStore;\n\n\n        public JoinStreamAggregateProcessor(String name, JoinType joinType, ValueJoinAction<V1, V2, OUT> joinAction) {\n            this.name = name;\n            this.joinType = joinType;\n            this.joinAction = joinAction;\n        }\n\n        @Override\n        public void preProcess(StreamContext<Object> context) throws RecoverStateStoreThrowable {\n            super.preProcess(context);\n            this.stateStore = super.waitStateReplay();\n            String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX;\n            this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId());\n        }\n\n        @Override\n        public void process(Object data) throws Throwable {\n            Object key = this.context.getKey();\n            Properties header = this.context.getHeader();\n            StreamType streamType = (StreamType) header.get(Constant.STREAM_TAG);\n\n            store(key, data, streamType);\n            fire(key, data, streamType);\n        }\n\n        private void store(Object key, Object data, StreamType streamType) throws Throwable {\n            String name = Utils.buildKey(this.name, streamType.name());\n\n            switch (streamType) {\n                case LEFT_STREAM:\n                case RIGHT_STREAM: {\n                    String storeKey = Utils.buildKey(name, super.toHexString(key));\n                    byte[] keyBytes = Utils.object2Byte(storeKey);\n                    byte[] valueBytes = super.object2Byte(data);\n\n                    this.stateStore.put(stateTopicMessageQueue, ColumnFamilyUtil.VALUE_STATE_CF, keyBytes, valueBytes);\n                    break;\n                }\n            }\n\n        }\n\n        @SuppressWarnings(\"unchecked\")\n        private void fire(Object key, Object data, StreamType streamType) throws Throwable {\n            switch (streamType) {\n                case LEFT_STREAM: {\n                    String name = Utils.buildKey(this.name, StreamType.RIGHT_STREAM.name());\n                    String storeKey = Utils.buildKey(name, super.toHexString(key));\n                    byte[] keyBytes = Utils.object2Byte(storeKey);\n\n                    byte[] bytes = this.stateStore.get(ColumnFamilyUtil.VALUE_STATE_CF, keyBytes);\n\n                    if (joinType == JoinType.INNER_JOIN) {\n                        if (bytes == null || bytes.length == 0) {\n                            break;\n                        }\n                    } else if (joinType == JoinType.LEFT_JOIN) {\n                        //no-op\n                    } else {\n                        throw new UnsupportedOperationException(\"unknown joinType = \" + joinType);\n                    }\n\n                    V1 v1Data = (V1) data;\n                    V2 v2Data = super.byte2Object(bytes);\n\n                    doFire(v1Data, v2Data);\n                    break;\n                }\n                case RIGHT_STREAM: {\n                    if (joinType != JoinType.INNER_JOIN) {\n                        break;\n                    }\n\n                    String name = Utils.buildKey(this.name, StreamType.LEFT_STREAM.name());\n                    String storeKey = Utils.buildKey(name, super.toHexString(key));\n                    byte[] keyBytes = Utils.object2Byte(storeKey);\n\n                    byte[] bytes = this.stateStore.get(ColumnFamilyUtil.VALUE_STATE_CF, keyBytes);\n                    if (bytes == null || bytes.length == 0) {\n                        break;\n                    }\n\n                    V2 v2Data = (V2) data;\n                    V1 v1Data = super.byte2Object(bytes);\n\n                    doFire(v1Data, v2Data);\n                    break;\n                }\n            }\n\n            //todo 是否需要删除状态？\n        }\n\n        private void doFire(V1 v1Data, V2 v2Data) throws Throwable {\n            OUT out = this.joinAction.apply(v1Data, v2Data);\n\n            Data<K, OUT> result = new Data<>(this.context.getKey(), out, this.context.getDataTime(), this.context.getHeader());\n            Data<K, Object> convert = super.convert(result);\n            this.context.forward(convert);\n        }\n    }\n\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/JoinWindowAggregateSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RStreamsException;\nimport org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable;\nimport org.apache.rocketmq.streams.core.function.ValueJoinAction;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractWindowProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.window.JoinType;\nimport org.apache.rocketmq.streams.core.window.StreamType;\nimport org.apache.rocketmq.streams.core.window.Window;\nimport org.apache.rocketmq.streams.core.window.WindowInfo;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.window.WindowState;\nimport org.apache.rocketmq.streams.core.window.WindowStore;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.streams.core.window.fire.JoinWindowFire;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.Comparator;\nimport java.util.List;\nimport java.util.Properties;\nimport java.util.concurrent.atomic.AtomicReference;\nimport java.util.function.Supplier;\n\npublic class JoinWindowAggregateSupplier<K, V1, V2, OUT> implements Supplier<Processor<? super OUT>> {\n    private static final Logger logger = LoggerFactory.getLogger(JoinWindowAggregateSupplier.class.getName());\n\n    private String name;\n    private WindowInfo windowInfo;\n    private final ValueJoinAction<V1, V2, OUT> joinAction;\n    private JoinType joinType;\n\n    public JoinWindowAggregateSupplier(String name, WindowInfo windowInfo, ValueJoinAction<V1, V2, OUT> joinAction) {\n        this.name = name;\n        this.windowInfo = windowInfo;\n        this.joinType = windowInfo.getJoinStream().getJoinType();\n        this.joinAction = joinAction;\n    }\n\n    @Override\n    public Processor<Object> get() {\n        return new JoinStreamWindowAggregateProcessor(name, windowInfo, joinType, joinAction);\n    }\n\n\n    @SuppressWarnings(\"unchecked\")\n    private class JoinStreamWindowAggregateProcessor extends AbstractWindowProcessor<Object> {\n        private String name;\n        private final WindowInfo windowInfo;\n        private final JoinType joinType;\n        private ValueJoinAction<V1, V2, OUT> joinAction;\n        private MessageQueue stateTopicMessageQueue;\n        private WindowStore<K, V1> leftWindowStore;\n        private WindowStore<K, V2> rightWindowStore;\n\n        public JoinStreamWindowAggregateProcessor(String name, WindowInfo windowInfo, JoinType joinType, ValueJoinAction<V1, V2, OUT> joinAction) {\n            this.name = Utils.buildKey(name, JoinStreamWindowAggregateProcessor.class.getSimpleName());\n            this.windowInfo = windowInfo;\n            this.joinType = joinType;\n            this.joinAction = joinAction;\n        }\n\n        @Override\n        public void preProcess(StreamContext<Object> context) throws RecoverStateStoreThrowable {\n            super.preProcess(context);\n            leftWindowStore = new WindowStore<>(super.waitStateReplay(), WindowState::byte2WindowState, WindowState::windowState2Byte);\n            rightWindowStore = new WindowStore<>(super.waitStateReplay(), WindowState::byte2WindowState, WindowState::windowState2Byte);\n\n            this.idleWindowScaner = context.getDefaultWindowScaner();\n\n            String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX;\n            this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId());\n\n            this.joinWindowFire = new JoinWindowFire<>(joinType,\n                    this.stateTopicMessageQueue,\n                    context.copy(),\n                    joinAction,\n                    leftWindowStore,\n                    rightWindowStore,\n                    this::watermark);\n        }\n\n        @Override\n        public void process(Object data) throws Throwable {\n            Object key = this.context.getKey();\n            long time = this.context.getDataTime();\n            Properties header = this.context.getHeader();\n\n            long watermark = this.watermark(time - allowDelay, stateTopicMessageQueue);\n\n            if (time < watermark) {\n                logger.warn(\"discard delay data:[{}]. time of data:{}, watermark:{}\", data, time, watermark);\n                return;\n            }\n            WindowInfo.JoinStream stream = (WindowInfo.JoinStream) header.get(Constant.STREAM_TAG);\n            StreamType streamType = stream.getStreamType();\n            if (streamType == null) {\n                String format = String.format(\"StreamType is empty, data:%s\", data);\n                throw new IllegalStateException(format);\n            }\n\n            store(key, data, time, watermark, streamType);\n\n            List<WindowKey> fire = this.joinWindowFire.fire(this.name, watermark, streamType);\n            for (WindowKey windowKey : fire) {\n                this.idleWindowScaner.removeWindowKey(windowKey);\n            }\n        }\n\n\n        private void store(Object key, Object data, long time, long watermark, StreamType streamType) throws Throwable {\n            String name = Utils.buildKey(this.name, streamType.name());\n            List<Window> windows = super.calculateWindow(windowInfo, time);\n            for (Window window : windows) {\n                logger.debug(\"timestamp=\" + time + \". time -> window: \" + Utils.format(time) + \"->\" + window);\n\n                WindowKey windowKey = new WindowKey(name, super.toHexString(key), window.getEndTime(), window.getStartTime());\n\n                switch (streamType) {\n                    case LEFT_STREAM:\n                        WindowState<K, V1> leftState = new WindowState<>((K) key, (V1) data, time);\n                        this.leftWindowStore.put(stateTopicMessageQueue, windowKey, leftState);\n                        this.idleWindowScaner.putJoinWindowCallback(windowKey, watermark, joinWindowFire);\n                        break;\n                    case RIGHT_STREAM:\n                        WindowState<K, V2> rightState = new WindowState<>((K) key, (V2) data, time);\n                        this.rightWindowStore.put(stateTopicMessageQueue, windowKey, rightState);\n                        this.idleWindowScaner.putJoinWindowCallback(windowKey, watermark, joinWindowFire);\n                        break;\n                }\n            }\n\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/KeySelectSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\nimport org.apache.rocketmq.streams.core.function.SelectAction;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\n\nimport java.util.function.Supplier;\n\npublic class KeySelectSupplier<KEY, T> implements Supplier<Processor<T>> {\n    private final SelectAction<KEY, T> selectAction;\n\n    public KeySelectSupplier(SelectAction<KEY, T> selectAction) {\n        this.selectAction = selectAction;\n    }\n\n    @Override\n    public Processor<T> get() {\n        return new MapperProcessor(selectAction);\n    }\n\n    private class MapperProcessor extends AbstractProcessor<T> {\n        private final SelectAction<KEY, T> selectAction;\n\n\n        public MapperProcessor(SelectAction<KEY, T> selectAction) {\n            this.selectAction = selectAction;\n        }\n\n        @Override\n        public void process(T data) throws Throwable {\n            KEY newKey = selectAction.select(data);\n            Data<KEY, T> result = new Data<>(newKey, data, this.context.getDataTime(), this.context.getHeader());\n            this.context.forward(result);\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/MultiValueChangeSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\n\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.function.Supplier;\n\npublic class MultiValueChangeSupplier<T, VR> implements Supplier<Processor<T>> {\n    private final ValueMapperAction<T, ? extends Iterable<? extends VR>> valueMapperAction;\n    private static final Logger logger = LoggerFactory.getLogger(MultiValueChangeSupplier.class.getName());\n\n    public MultiValueChangeSupplier(ValueMapperAction<T, ? extends Iterable<? extends VR>> valueMapperAction) {\n        this.valueMapperAction = valueMapperAction;\n    }\n\n    @Override\n    public Processor<T> get() {\n        return new MultiValueMapperProcessor<>(this.valueMapperAction);\n    }\n\n    static class MultiValueMapperProcessor<T, VR> extends AbstractProcessor<T> {\n        private final ValueMapperAction<T, ? extends Iterable<? extends VR>> valueMapperAction;\n\n        public MultiValueMapperProcessor(ValueMapperAction<T, ? extends Iterable<? extends VR>> valueMapperAction) {\n            this.valueMapperAction = valueMapperAction;\n        }\n\n        @Override\n        public void process(T data) throws Throwable {\n            Iterable<? extends VR> convert = valueMapperAction.convert(data);\n\n            if (convert == null) {\n                logger.warn(\"[{}] converts to null, processor returns directly\", data);\n                return;\n            }\n\n            for (VR item : convert) {\n                if (item == null) {\n                    continue;\n                }\n                Data<Object, VR> before = new Data<>(this.context.getKey(), item, this.context.getDataTime(), this.context.getHeader());\n                Data<Object, T> result = convert(before);\n                this.context.forward(result);\n            }\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/PrintSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.util.Utils;\n\nimport java.util.Properties;\nimport java.util.function.Supplier;\n\npublic class PrintSupplier<T> implements Supplier<Processor<T>> {\n\n\n    @Override\n    public Processor<T> get() {\n        return new PrintProcessor<>();\n    }\n\n    static class PrintProcessor<T> extends AbstractProcessor<T> {\n\n\n        public PrintProcessor() {\n        }\n\n\n        @Override\n        public void process(T data) {\n            Properties header = context.getHeader();\n            Object startTime = header.get(Constant.WINDOW_START_TIME);\n            Object endTime = header.get(Constant.WINDOW_END_TIME);\n            if (startTime == null || endTime == null) {\n                String template = \"(key=%s, value=%s)\";\n\n                Data<Object, T> result = new Data<>(this.context.getKey(), data, this.context.getDataTime(), header);\n                String format = String.format(template, result.getKey(), data.toString());\n\n                System.out.println(format);\n            } else {\n                String template = \"[%s - %s](key=%s, value=%s)\";\n                String start = Utils.format((Long)startTime);\n                String end = Utils.format((Long)endTime);\n\n                String format = String.format(template, start, end, this.context.getKey(), data);\n                System.out.println(format);\n            }\n\n\n        }\n    }\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SinkSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\n\nimport org.apache.rocketmq.client.producer.DefaultMQProducer;\nimport org.apache.rocketmq.client.producer.selector.SelectMessageQueueByHash;\nimport org.apache.rocketmq.common.message.Message;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueSerializer;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.function.Supplier;\n\npublic class SinkSupplier<K, T> implements Supplier<Processor<T>> {\n    private static final Logger logger = LoggerFactory.getLogger(SinkSupplier.class);\n\n    private final String topicName;\n    private final KeyValueSerializer<K, T> serializer;\n\n    public SinkSupplier(String topicName, KeyValueSerializer<K, T> serializer) {\n        this.topicName = topicName;\n        this.serializer = serializer;\n    }\n\n    @Override\n    public Processor<T> get() {\n        return new SinkProcessor(this.topicName, this.serializer);\n    }\n\n    private class SinkProcessor extends AbstractProcessor<T> {\n        private final String topicName;\n        private DefaultMQProducer producer;\n        private final KeyValueSerializer<K, T> serializer;\n        private K key;\n\n        public SinkProcessor(String topicName, KeyValueSerializer<K, T> serializer) {\n            this.topicName = topicName;\n            this.serializer = serializer;\n        }\n\n        @Override\n        public void preProcess(StreamContext<T> context) {\n            this.context = context;\n            this.producer = context.getDefaultMQProducer();\n            this.key = context.getKey();\n        }\n\n        //sink into shuffle topic/state topic/user topic\n        @Override\n        public void process(T data) throws Throwable {\n            if (data != null) {\n                byte[] value = this.serializer.serialize(key, data);\n                if (value == null || value.length == 0) {\n                    //目前RocketMQ不支持发送body为null的消息；\n                    return;\n                }\n\n                Message message;\n\n                if (this.key == null) {\n                    message = new Message(this.topicName, value);\n                    message.putUserProperty(Constant.SHUFFLE_VALUE_CLASS_NAME, data.getClass().getName());\n                    if (this.topicName.contains(Constant.SHUFFLE_TOPIC_SUFFIX)) {\n                        message.putUserProperty(Constant.SOURCE_TIMESTAMP, String.valueOf(this.context.getDataTime()));\n                    }\n\n                    producer.send(message);\n                } else {\n                    message = new Message(this.topicName, value);\n                    String hexKey = Utils.toHexString(this.key);\n                    //the real key is in the body, this key is used to route the same key into the same queue.\n                    message.setKeys(hexKey);\n\n\n                    message.putUserProperty(Constant.SHUFFLE_KEY_CLASS_NAME, this.key.getClass().getName());\n                    message.putUserProperty(Constant.SHUFFLE_VALUE_CLASS_NAME, data.getClass().getName());\n\n                    if (this.topicName.contains(Constant.SHUFFLE_TOPIC_SUFFIX)) {\n                        message.putUserProperty(Constant.SOURCE_TIMESTAMP, String.valueOf(this.context.getDataTime()));\n                    }\n\n                    producer.send(message, new SelectMessageQueueByHash(), hexKey);\n                }\n            }\n        }\n\n\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SourceSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.common.message.MessageExt;\nimport org.apache.rocketmq.streams.core.exception.DeserializeThrowable;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.window.TimeType;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueDeserializer;\nimport org.apache.rocketmq.streams.core.util.Pair;\n\nimport java.util.function.Supplier;\n\npublic class SourceSupplier<K, V> implements Supplier<Processor<V>> {\n    private String topicName;\n    private KeyValueDeserializer<K, V> deserializer;\n\n    public SourceSupplier(String topicName, KeyValueDeserializer<K, V> deserializer) {\n        this.topicName = topicName;\n        this.deserializer = deserializer;\n    }\n\n    @Override\n    public Processor<V> get() {\n        return new SourceProcessorImpl(deserializer);\n    }\n\n    public interface SourceProcessor<K, V> extends Processor<V> {\n        Pair<K, V> deserialize(String keyClass, String valueClass, byte[] data) throws DeserializeThrowable;\n\n        long getTimestamp(MessageExt originData, TimeType timeType);\n    }\n\n    private class SourceProcessorImpl extends AbstractProcessor<V> implements SourceProcessor<K, V> {\n        private KeyValueDeserializer<K, V> deserializer;\n\n        public SourceProcessorImpl(KeyValueDeserializer<K, V> deserializer) {\n            this.deserializer = deserializer;\n        }\n\n        @Override\n        public Pair<K, V> deserialize(String keyClass, String valueClass, byte[] data) throws DeserializeThrowable {\n            try {\n                this.deserializer.configure(keyClass, valueClass);\n                return this.deserializer.deserialize(data);\n            }catch (Throwable t) {\n                throw new DeserializeThrowable(t);\n            }\n        }\n\n        @Override\n        public long getTimestamp(MessageExt originData, TimeType timeType) {\n            if (timeType == null) {\n                return System.currentTimeMillis();\n            } else if (timeType == TimeType.EVENT_TIME) {\n                return originData.getBornTimestamp();\n            } else if (timeType == TimeType.PROCESS_TIME) {\n                return System.currentTimeMillis();\n            } else {\n                throw new IllegalStateException(\"unknown time type: \" + timeType.getClass().getName());\n            }\n        }\n\n\n        @Override\n        public void process(V data) throws Throwable {\n            //no-op\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/SumAggregate.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\n\nimport org.apache.rocketmq.streams.core.function.AggregateAction;\nimport org.apache.rocketmq.streams.core.function.SelectAction;\n\npublic class SumAggregate<K, V> implements AggregateAction<K, V, Number> {\n    private final SelectAction<? extends Number, V> selectAction;\n\n    public SumAggregate(SelectAction<? extends Number, V> selectAction) {\n        this.selectAction = selectAction;\n    }\n\n    @Override\n    public Number calculate(K key, V value, Number accumulator) {\n        Number number = selectAction.select(value);\n        if (accumulator == null) {\n            accumulator = number;\n            return accumulator;\n        }\n\n        if (number instanceof Integer) {\n            return accumulator.intValue() + number.intValue();\n        } else if (number instanceof Long) {\n            return accumulator.longValue() + number.longValue();\n        } else if (number instanceof Double) {\n            return accumulator.doubleValue() + number.doubleValue();\n        } else if (number instanceof Float) {\n            return accumulator.floatValue() + number.floatValue();\n        } else {\n            throw new UnsupportedOperationException(\"unsupported number type:\" + number.getClass().getName());\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/TimestampSelectorSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\n\nimport java.util.function.Supplier;\n\npublic class TimestampSelectorSupplier<T> implements Supplier<Processor<T>> {\n    private final ValueMapperAction<T, Long> valueMapperAction;\n\n    public TimestampSelectorSupplier(ValueMapperAction<T, Long> valueMapperAction) {\n        this.valueMapperAction = valueMapperAction;\n    }\n\n    @Override\n    public Processor<T> get() {\n        return new TimestampSelector<>(valueMapperAction);\n    }\n\n\n    static class TimestampSelector<T> extends AbstractProcessor<T> {\n        private final ValueMapperAction<T, Long> valueMapperAction;\n\n        public TimestampSelector(ValueMapperAction<T, Long> valueMapperAction) {\n            this.valueMapperAction = valueMapperAction;\n        }\n\n        @Override\n        public void process(T data) throws Throwable {\n            Long timestamp = this.valueMapperAction.convert(data);\n\n            StreamContext<T> streamContext = this.context;\n\n            //override the timestamp of data\n            Data<Object, T> result = new Data<>(streamContext.getKey(), data, timestamp, streamContext.getHeader());\n            streamContext.forward(result);\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/ValueChangeSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\n\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.AbstractProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.function.Supplier;\n\npublic class ValueChangeSupplier<T, O> implements Supplier<Processor<T>> {\n    private final ValueMapperAction<T, O> valueMapperAction;\n    private static final Logger logger = LoggerFactory.getLogger(ValueChangeSupplier.class.getName());\n\n\n    public ValueChangeSupplier(ValueMapperAction<T, O> valueMapperAction) {\n        this.valueMapperAction = valueMapperAction;\n    }\n\n    @Override\n    public Processor<T> get() {\n        return new ValueMapperProcessor<>(this.valueMapperAction);\n    }\n\n\n    static class ValueMapperProcessor<T, O> extends AbstractProcessor<T> {\n        private final ValueMapperAction<T, O> valueMapperAction;\n\n\n        public ValueMapperProcessor(ValueMapperAction<T, O> valueMapperAction) {\n            this.valueMapperAction = valueMapperAction;\n        }\n\n        @Override\n        public void process(T data) throws Throwable {\n            O convert = valueMapperAction.convert(data);\n            if (convert == null) {\n                logger.warn(\"[{}] converts to null, processor returns directly\", data);\n                return;\n            }\n            Data<Object, O> before = new Data<>(this.context.getKey(), convert, this.context.getDataTime(), this.context.getHeader());\n            Data<Object, T> result = convert(before);\n            this.context.forward(result);\n        }\n    }\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/WindowAccumulatorSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable;\nimport org.apache.rocketmq.streams.core.function.SelectAction;\nimport org.apache.rocketmq.streams.core.function.accumulator.Accumulator;\nimport org.apache.rocketmq.streams.core.running.AbstractWindowProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.window.Window;\nimport org.apache.rocketmq.streams.core.window.WindowInfo;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.window.WindowState;\nimport org.apache.rocketmq.streams.core.window.WindowStore;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.streams.core.window.fire.AccumulatorWindowFire;\nimport org.apache.rocketmq.streams.core.window.fire.AccumulatorSessionWindowFire;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.Iterator;\nimport java.util.List;\nimport java.util.concurrent.atomic.AtomicReference;\nimport java.util.function.Supplier;\n\npublic class WindowAccumulatorSupplier<K, V, R, OV> implements Supplier<Processor<V>> {\n    private static final Logger logger = LoggerFactory.getLogger(WindowAccumulatorSupplier.class.getName());\n    private final String name;\n    private WindowInfo windowInfo;\n    private SelectAction<R, V> selectAction;\n    private Accumulator<R, OV> accumulator;\n\n    public WindowAccumulatorSupplier(String name, WindowInfo windowInfo,\n                                     SelectAction<R, V> selectAction, Accumulator<R, OV> accumulator) {\n        this.name = name;\n        this.windowInfo = windowInfo;\n        this.selectAction = selectAction;\n        this.accumulator = accumulator;\n    }\n\n    @Override\n    public Processor<V> get() {\n        WindowInfo.WindowType windowType = windowInfo.getWindowType();\n        switch (windowType) {\n            case SLIDING_WINDOW:\n            case TUMBLING_WINDOW:\n                return new WindowAccumulatorProcessor(name, windowInfo, selectAction, accumulator);\n            case SESSION_WINDOW:\n                return new SessionWindowAccumulatorProcessor(name, windowInfo, selectAction, accumulator);\n            default:\n                throw new RuntimeException(\"window type is error, WindowType=\" + windowType);\n        }\n    }\n\n\n    public class WindowAccumulatorProcessor extends AbstractWindowProcessor<V> {\n        private final WindowInfo windowInfo;\n        private String name;\n        private MessageQueue stateTopicMessageQueue;\n        private SelectAction<R, V> selectAction;\n        private Accumulator<R, OV> accumulator;\n        private WindowStore<K, Accumulator<R, OV>> windowStore;\n\n        private final AtomicReference<Throwable> errorReference = new AtomicReference<>(null);\n\n        public WindowAccumulatorProcessor(String name, WindowInfo windowInfo, SelectAction<R, V> selectAction, Accumulator<R, OV> accumulator) {\n            this.name = String.join(Constant.SPLIT, name, WindowAccumulatorProcessor.class.getSimpleName());\n            this.windowInfo = windowInfo;\n            this.selectAction = selectAction;\n            this.accumulator = accumulator;\n        }\n\n        @Override\n        public void preProcess(StreamContext<V> context) throws RecoverStateStoreThrowable {\n            super.preProcess(context);\n            this.windowStore = new WindowStore<>(super.waitStateReplay(),\n                    WindowState::byte2WindowState,\n                    WindowState::windowState2Byte);\n\n            this.idleWindowScaner = context.getDefaultWindowScaner();\n\n            String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX;\n            this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId());\n\n            this.accumulatorWindowFire = new AccumulatorWindowFire<>(this.windowStore,\n                    context.copy(),\n                    this.stateTopicMessageQueue,\n                    this::watermark);\n        }\n\n        /**\n         * 维持一个watermark，小于watermark的数据都已经达到，触发窗口计算\n         */\n        @Override\n        public void process(V data) throws Throwable {\n            Throwable throwable = errorReference.get();\n            if (throwable != null) {\n                errorReference.set(null);\n                throw throwable;\n            }\n\n            K key = this.context.getKey();\n            long time = this.context.getDataTime();\n\n            long watermark = this.watermark(time - allowDelay, stateTopicMessageQueue);\n            if (time < watermark) {\n                //delay data.\n                logger.warn(\"discard delay data:[{}]. time of data:{}, watermark:{}\", data, time, watermark);\n                return;\n            }\n\n            //f(time) -> List<Window>\n            List<Window> windows = super.calculateWindow(windowInfo, time);\n            for (Window window : windows) {\n                logger.debug(\"timestamp=\" + time + \". time -> window: \" + Utils.format(time) + \"->\" + window);\n\n                //f(Window + key, store) -> oldValue\n                //todo key 怎么转化成对应的string，只和key的值有关系\n                WindowKey windowKey = new WindowKey(name, super.toHexString(key), window.getEndTime(), window.getStartTime());\n                WindowState<K, Accumulator<R, OV>> oldState = this.windowStore.get(windowKey);\n\n                //f(oldValue, Agg) -> newValue\n                Accumulator<R, OV> storeAccumulator;\n                if (oldState == null || oldState.getValue() == null) {\n                    storeAccumulator = accumulator.clone();\n                } else {\n                    storeAccumulator = oldState.getValue();\n                }\n\n                R select = selectAction.select(data);\n                storeAccumulator.addValue(select);\n\n                //f(Window + key, newValue, store)\n                WindowState<K, Accumulator<R, OV>> state = new WindowState<>(key, storeAccumulator, time);\n                this.windowStore.put(stateTopicMessageQueue, windowKey, state);\n                this.idleWindowScaner.putAccumulatorWindowCallback(windowKey, watermark, this.accumulatorWindowFire);\n            }\n\n            try {\n                List<WindowKey> fire = this.accumulatorWindowFire.fire(name, watermark);\n                for (WindowKey windowKey : fire) {\n                    this.idleWindowScaner.removeWindowKey(windowKey);\n                }\n            } catch (Throwable t) {\n                errorReference.compareAndSet(null, t);\n            }\n        }\n    }\n\n    private class SessionWindowAccumulatorProcessor extends AbstractWindowProcessor<V> {\n        private final String name;\n        private final WindowInfo windowInfo;\n        private MessageQueue stateTopicMessageQueue;\n        private SelectAction<R, V> selectAction;\n        private Accumulator<R, OV> accumulator;\n        private WindowStore<K, Accumulator<R, OV>> windowStore;\n\n        public SessionWindowAccumulatorProcessor(String name, WindowInfo windowInfo, SelectAction<R, V> selectAction, Accumulator<R, OV> accumulator) {\n            this.name = String.join(Constant.SPLIT, name, SessionWindowAccumulatorProcessor.class.getSimpleName());\n            this.windowInfo = windowInfo;\n            this.selectAction = selectAction;\n            this.accumulator = accumulator;\n        }\n\n        @Override\n        public void preProcess(StreamContext<V> context) throws RecoverStateStoreThrowable {\n            super.preProcess(context);\n            this.windowStore = new WindowStore<>(super.waitStateReplay(),\n                    WindowState::byte2WindowState,\n                    WindowState::windowState2Byte);\n\n            this.idleWindowScaner = context.getDefaultWindowScaner();\n\n            String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX;\n            this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId());\n\n            this.accumulatorSessionWindowFire = new AccumulatorSessionWindowFire<>(this.windowStore,\n                    context.copy(),\n                    this.stateTopicMessageQueue,\n                    this::watermark);\n        }\n\n        @Override\n        public void process(V data) throws Throwable {\n            K key = this.context.getKey();\n            long time = this.context.getDataTime();\n\n            long watermark = this.watermark(time - allowDelay, stateTopicMessageQueue);\n            if (time < watermark) {\n                logger.warn(\"discard delay data:[{}]. time of data:{}, watermark:{}\", data, time, watermark);\n                return;\n            }\n            //本地存储里面搜索下\n            Pair<Long, Long> newSessionWindowTime = fireIfSessionOut(key, data, time, watermark);\n\n            if (newSessionWindowTime != null) {\n                Accumulator<R, OV> temp = accumulator.clone();\n                R select = selectAction.select(data);\n                temp.addValue(select);\n\n                WindowState<K, Accumulator<R, OV>> state = new WindowState<>(key, temp, time);\n                if (time < state.getRecordEarliestTimestamp()) {\n                    //更新最早时间戳，用于状态触发时候，作为session 窗口的begin时间戳\n                    state.setRecordEarliestTimestamp(time);\n                }\n\n                WindowKey windowKey = new WindowKey(name, super.toHexString(key), newSessionWindowTime.getValue(), newSessionWindowTime.getKey());\n                logger.info(\"new session window, with key={}, valueTime={}, sessionBegin=[{}], sessionEnd=[{}]\", key, Utils.format(time),\n                        Utils.format(newSessionWindowTime.getKey()), Utils.format(newSessionWindowTime.getValue()));\n                this.windowStore.put(stateTopicMessageQueue, windowKey, state);\n                this.idleWindowScaner.putAccumulatorSessionWindowCallback(windowKey, watermark, this.accumulatorSessionWindowFire);\n            }\n        }\n\n\n        //使用前缀查询找到session state, 触发已经session out的 watermark\n        @SuppressWarnings(\"unchecked\")\n        private Pair<Long/*sessionBegin*/, Long/*sessionEnd*/> fireIfSessionOut(K key, V data, long dataTime, long watermark) throws Throwable {\n            List<Pair<WindowKey, WindowState<K, Accumulator<R, OV>>>> pairs = this.windowStore.searchMatchKeyPrefix(name);\n\n            if (pairs.size() == 0) {\n                return new Pair<>(dataTime, dataTime + windowInfo.getSessionTimeout().toMilliseconds());\n            }\n\n            logger.debug(\"exist session state num={}\", pairs.size());\n\n            //sessionEndTime小的先触发\n            Iterator<Pair<WindowKey, WindowState<K, Accumulator<R, OV>>>> iterator = pairs.iterator();\n            int count = 0;\n            long lastStateSessionEnd = 0;\n            long maxFireSessionEnd = Long.MIN_VALUE;\n\n            while (iterator.hasNext()) {\n                Pair<WindowKey, WindowState<K, Accumulator<R, OV>>> pair = iterator.next();\n                logger.debug(\"exist session state{}=[{}]\", count++, pair);\n\n                WindowKey windowKey = pair.getKey();\n\n                long sessionEnd = windowKey.getWindowEnd();\n                if (count == pairs.size()) {\n                    lastStateSessionEnd = sessionEnd;\n                }\n\n                //先触发一遍，触发后从集合中删除\n                if (sessionEnd < watermark) {\n                    //触发state\n                    List<WindowKey> fire = this.accumulatorSessionWindowFire.fire(name, watermark);\n                    for (WindowKey delete : fire) {\n                        this.idleWindowScaner.removeWindowKey(delete);\n                    }\n\n                    iterator.remove();\n                    maxFireSessionEnd = Long.max(sessionEnd, maxFireSessionEnd);\n                }\n            }\n\n            if (dataTime < maxFireSessionEnd) {\n                logger.warn(\"late data, discard. key=[{}], data=[{}], dataTime < maxFireSessionEnd: [{}] < [{}]\", key, data, dataTime, maxFireSessionEnd);\n                return null;\n            }\n\n            boolean createNewSessionWindow = false;\n            WindowKey needToDelete = null;\n\n            //再次遍历，找到数据属于某个窗口，如果窗口已经关闭，则只计算新的值，如果窗口没有关闭则计算新值、更新窗口边界、存储状态、删除老值\n            for (int i = 0; i < pairs.size(); i++) {\n                Pair<WindowKey, WindowState<K, Accumulator<R, OV>>> pair = pairs.get(i);\n\n                WindowKey windowKey = pair.getKey();\n                WindowState<K, Accumulator<R, OV>> state = pair.getValue();\n\n\n                if (windowKey.getWindowEnd() < dataTime) {\n                    createNewSessionWindow = true;\n                } else if (windowKey.getWindowStart() <= dataTime) {\n                    logger.debug(\"data belong to exist session window.dataTime=[{}], window:[{} - {}]\", dataTime, Utils.format(windowKey.getWindowStart()), Utils.format(windowKey.getWindowEnd()));\n                    Accumulator<R, OV> value = state.getValue();\n\n                    R select = selectAction.select(data);\n                    value.addValue(select);\n\n                    //更新state\n                    state.setValue(value);\n                    state.setRecordLastTimestamp(dataTime);\n                    if (dataTime < state.getRecordEarliestTimestamp()) {\n                        //更新最早时间戳，用于状态触发时候，作为session 窗口的begin时间戳\n                        state.setRecordEarliestTimestamp(dataTime);\n                    }\n\n                    //如果是最后一个窗口，更新窗口结束时间\n                    if (i == pairs.size() - 1) {\n                        long mayBeSessionEnd = dataTime + windowInfo.getSessionTimeout().toMilliseconds();\n                        if (windowKey.getWindowEnd() < mayBeSessionEnd) {\n                            logger.debug(\"update exist session window, before:[{} - {}], after:[{} - {}]\", Utils.format(windowKey.getWindowStart()), Utils.format(windowKey.getWindowEnd()),\n                                    Utils.format(windowKey.getWindowStart()), Utils.format(mayBeSessionEnd));\n                            //删除老状态\n                            needToDelete = windowKey;\n                            //需要保存的新状态\n                            windowKey = new WindowKey(windowKey.getOperatorName(), windowKey.getKey2String(), mayBeSessionEnd, windowKey.getWindowStart());\n                        }\n                    }\n                } else {\n                    logger.warn(\"discard data: key=[{}], data=[{}], dataTime=[{}], watermark=[{}]\", key, data, dataTime, watermark);\n                }\n\n                this.windowStore.put(stateTopicMessageQueue, windowKey, state);\n\n                this.idleWindowScaner.putAccumulatorSessionWindowCallback(windowKey, watermark, this.accumulatorSessionWindowFire);\n                this.idleWindowScaner.removeOldAccumulatorSession(needToDelete);\n\n                this.windowStore.deleteByKey(needToDelete);\n            }\n\n            if (pairs.size() == 0 || createNewSessionWindow) {\n                return new Pair<>(lastStateSessionEnd, dataTime + windowInfo.getSessionTimeout().toMilliseconds());\n            }\n            return null;\n        }\n\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/WindowAggregateSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function.supplier;\n\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable;\nimport org.apache.rocketmq.streams.core.function.AggregateAction;\nimport org.apache.rocketmq.streams.core.running.AbstractWindowProcessor;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.streams.core.window.Window;\nimport org.apache.rocketmq.streams.core.window.WindowInfo;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.window.WindowState;\nimport org.apache.rocketmq.streams.core.window.WindowStore;\nimport org.apache.rocketmq.streams.core.window.fire.AggregateSessionWindowFire;\nimport org.apache.rocketmq.streams.core.window.fire.AggregateWindowFire;\nimport org.checkerframework.checker.units.qual.K;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.Iterator;\nimport java.util.List;\nimport java.util.concurrent.atomic.AtomicReference;\nimport java.util.function.Supplier;\n\npublic class WindowAggregateSupplier<K, V, OV> implements Supplier<Processor<V>> {\n    private static final Logger logger = LoggerFactory.getLogger(WindowAggregateSupplier.class.getName());\n    private String name;\n    private WindowInfo windowInfo;\n    private Supplier<OV> initAction;\n    private AggregateAction<K, V, OV> aggregateAction;\n\n    public WindowAggregateSupplier(String name, WindowInfo windowInfo, Supplier<OV> initAction, AggregateAction<K, V, OV> aggregateAction) {\n        this.name = name;\n        this.windowInfo = windowInfo;\n        this.initAction = initAction;\n        this.aggregateAction = aggregateAction;\n    }\n\n    @Override\n    public Processor<V> get() {\n        WindowInfo.WindowType windowType = windowInfo.getWindowType();\n        switch (windowType) {\n            case SLIDING_WINDOW:\n            case TUMBLING_WINDOW:\n                return new WindowAggregateProcessor(name, windowInfo, initAction, aggregateAction);\n            case SESSION_WINDOW:\n                return new SessionWindowAggregateProcessor(name, windowInfo, initAction, aggregateAction);\n            default:\n                throw new RuntimeException(\"window type is error, WindowType=\" + windowType);\n        }\n    }\n\n\n    private class WindowAggregateProcessor extends AbstractWindowProcessor<V> {\n        private final WindowInfo windowInfo;\n        private String name;\n        private Supplier<OV> initAction;\n        private AggregateAction<K, V, OV> aggregateAction;\n        private MessageQueue stateTopicMessageQueue;\n        private WindowStore<K, OV> windowStore;\n\n        private final AtomicReference<Throwable> errorReference = new AtomicReference<>(null);\n\n        public WindowAggregateProcessor(String name, WindowInfo windowInfo, Supplier<OV> initAction, AggregateAction<K, V, OV> aggregateAction) {\n            this.name = String.join(Constant.SPLIT, name, WindowAggregateProcessor.class.getSimpleName());\n            this.windowInfo = windowInfo;\n            this.initAction = initAction;\n            this.aggregateAction = aggregateAction;\n        }\n\n        @Override\n        public void preProcess(StreamContext<V> context) throws RecoverStateStoreThrowable {\n            super.preProcess(context);\n            this.windowStore = new WindowStore<>(super.waitStateReplay(),\n                    WindowState::byte2WindowState,\n                    WindowState::windowState2Byte);\n\n            this.idleWindowScaner = context.getDefaultWindowScaner();\n\n            String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX;\n            this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId());\n\n            this.aggregateWindowFire = new AggregateWindowFire<>(this.windowStore,\n                    this.stateTopicMessageQueue,\n                    context.copy(),\n                    this::watermark);\n        }\n\n        /**\n         * 维持一个watermark，小于watermark的数据都已经达到，触发窗口计算\n         */\n        @Override\n        public void process(V data) throws Throwable {\n            Throwable throwable = errorReference.get();\n            if (throwable != null) {\n                errorReference.set(null);\n                throw throwable;\n            }\n\n            K key = this.context.getKey();\n            long time = this.context.getDataTime();\n\n            long watermark = this.watermark(time - allowDelay, stateTopicMessageQueue);\n            if (time < watermark) {\n                logger.warn(\"discard delay data:[{}]. time of data:{}, watermark:{}\", data, time, watermark);\n                return;\n            }\n\n\n            //f(time) -> List<Window>\n            List<Window> windows = super.calculateWindow(windowInfo, time);\n            for (Window window : windows) {\n                logger.debug(\"timestamp=\" + time + \". time -> window: \" + Utils.format(time) + \"->\" + window);\n\n                //f(Window + key, store) -> oldValue\n                //todo key 怎么转化成对应的string，只和key的值有关系\n                WindowKey windowKey = new WindowKey(name, super.toHexString(key), window.getEndTime(), window.getStartTime());\n                WindowState<K, OV> oldState = this.windowStore.get(windowKey);\n\n                //f(oldValue, Agg) -> newValue\n                OV oldValue;\n                if (oldState == null || oldState.getValue() == null) {\n                    oldValue = initAction.get();\n                } else {\n                    oldValue = oldState.getValue();\n                }\n\n                OV newValue = this.aggregateAction.calculate(key, data, oldValue);\n                if (newValue != null && newValue.equals(oldValue)) {\n                    continue;\n                }\n\n                //f(Window + key, newValue, store)\n                WindowState<K, OV> state = new WindowState<>(key, newValue, time);\n                this.windowStore.put(stateTopicMessageQueue, windowKey, state);\n                this.idleWindowScaner.putAggregateWindowCallback(windowKey, watermark, this.aggregateWindowFire);\n            }\n\n            try {\n                List<WindowKey> fire = this.aggregateWindowFire.fire(name, watermark);\n                for (WindowKey windowKey : fire) {\n                    this.idleWindowScaner.removeWindowKey(windowKey);\n                }\n            } catch (Throwable t) {\n                errorReference.compareAndSet(null, t);\n            }\n        }\n    }\n\n\n    private class SessionWindowAggregateProcessor extends AbstractWindowProcessor<V> {\n        private final String name;\n        private final WindowInfo windowInfo;\n        private Supplier<OV> initAction;\n        private AggregateAction<K, V, OV> aggregateAction;\n        private MessageQueue stateTopicMessageQueue;\n        private WindowStore<K, OV> windowStore;\n\n\n        public SessionWindowAggregateProcessor(String name, WindowInfo windowInfo, Supplier<OV> initAction, AggregateAction<K, V, OV> aggregateAction) {\n            this.name = String.join(Constant.SPLIT, name, SessionWindowAggregateProcessor.class.getSimpleName());\n            this.windowInfo = windowInfo;\n            this.initAction = initAction;\n            this.aggregateAction = aggregateAction;\n        }\n\n        @Override\n        public void preProcess(StreamContext<V> context) throws RecoverStateStoreThrowable {\n            super.preProcess(context);\n            this.windowStore = new WindowStore<>(super.waitStateReplay(),\n                    WindowState::byte2WindowState,\n                    WindowState::windowState2Byte);\n\n            this.idleWindowScaner = context.getDefaultWindowScaner();\n\n            String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX;\n            this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId());\n\n            this.aggregateSessionWindowFire = new AggregateSessionWindowFire<>(this.windowStore,\n                    this.stateTopicMessageQueue,\n                    context.copy(),\n                    this::watermark);\n        }\n\n        @Override\n        public void process(V data) throws Throwable {\n            K key = this.context.getKey();\n            long time = this.context.getDataTime();\n\n            long watermark = this.watermark(time - allowDelay, stateTopicMessageQueue);\n            if (time < watermark) {\n                logger.warn(\"discard delay data:[{}]. time of data:{}, watermark:{}\", data, time, watermark);\n                return;\n            }\n\n            //本地存储里面搜索下\n            Pair<Long, Long> newSessionWindowTime = fireIfSessionOut(key, data, time, watermark);\n\n            if (newSessionWindowTime != null) {\n                OV oldValue = this.initAction.get();\n                OV newValue = this.aggregateAction.calculate(key, data, oldValue);\n\n                WindowState<K, OV> state = new WindowState<>(key, newValue, time);\n                if (time < state.getRecordEarliestTimestamp()) {\n                    //更新最早时间戳，用于状态触发时候，作为session 窗口的begin时间戳\n                    state.setRecordEarliestTimestamp(time);\n                }\n\n                WindowKey windowKey = new WindowKey(name, super.toHexString(key), newSessionWindowTime.getValue(), newSessionWindowTime.getKey());\n                logger.info(\"new session window, with key={}, valueTime={}, sessionBegin=[{}], sessionEnd=[{}]\", key, Utils.format(time),\n                        Utils.format(newSessionWindowTime.getKey()), Utils.format(newSessionWindowTime.getValue()));\n                this.windowStore.put(stateTopicMessageQueue, windowKey, state);\n                this.idleWindowScaner.putAggregateSessionWindowCallback(windowKey, watermark, this.aggregateSessionWindowFire);\n            }\n        }\n\n\n        //使用前缀查询找到session state, 触发已经session out的 watermark\n        @SuppressWarnings(\"unchecked\")\n        private Pair<Long/*sessionBegin*/, Long/*sessionEnd*/> fireIfSessionOut(K key, V data, long dataTime, long watermark) throws Throwable {\n            List<Pair<WindowKey, WindowState<K, OV>>> pairs = this.windowStore.searchMatchKeyPrefix(name);\n\n            if (pairs.size() == 0) {\n                return new Pair<>(dataTime, dataTime + windowInfo.getSessionTimeout().toMilliseconds());\n            }\n\n            logger.debug(\"exist session state num={}\", pairs.size());\n\n            //sessionEndTime小的先触发\n            Iterator<Pair<WindowKey, WindowState<K, OV>>> iterator = pairs.iterator();\n            int count = 0;\n            long lastStateSessionEnd = 0;\n            long maxFireSessionEnd = Long.MIN_VALUE;\n\n            while (iterator.hasNext()) {\n                Pair<WindowKey, WindowState<K, OV>> pair = iterator.next();\n                logger.debug(\"exist session state{}=[{}]\", count++, pair);\n\n                WindowKey windowKey = pair.getKey();\n\n                long sessionEnd = windowKey.getWindowEnd();\n                if (count == pairs.size()) {\n                    lastStateSessionEnd = sessionEnd;\n                }\n\n                //先触发一遍，触发后从集合中删除\n                if (sessionEnd < watermark) {\n                    //触发state\n                    List<WindowKey> fire = this.aggregateSessionWindowFire.fire(name, watermark);\n                    for (WindowKey delete : fire) {\n                        this.idleWindowScaner.removeWindowKey(delete);\n                    }\n                    iterator.remove();\n                    maxFireSessionEnd = Long.max(sessionEnd, maxFireSessionEnd);\n                }\n            }\n\n            if (dataTime < maxFireSessionEnd) {\n                logger.warn(\"late data, discard. key=[{}], data=[{}], dataTime < maxFireSessionEnd: [{}] < [{}]\", key, data, dataTime, maxFireSessionEnd);\n                return null;\n            }\n\n            boolean createNewSessionWindow = false;\n            WindowKey needToDelete = null;\n\n            //再次遍历，找到数据属于某个窗口，如果窗口已经关闭，则只计算新的值，如果窗口没有关闭则计算新值、更新窗口边界、存储状态、删除老值\n            for (int i = 0; i < pairs.size(); i++) {\n                Pair<WindowKey, WindowState<K, OV>> pair = pairs.get(i);\n\n                WindowKey windowKey = pair.getKey();\n                WindowState<K, OV> state = pair.getValue();\n\n                if (windowKey.getWindowEnd() < dataTime) {\n                    createNewSessionWindow = true;\n                } else if (windowKey.getWindowStart() <= dataTime) {\n                    logger.debug(\"data belong to exist session window.dataTime=[{}], window:[{} - {}]\", dataTime, Utils.format(windowKey.getWindowStart()), Utils.format(windowKey.getWindowEnd()));\n                    OV newValue = this.aggregateAction.calculate(key, data, state.getValue());\n\n                    //更新state\n                    state.setValue(newValue);\n                    state.setRecordLastTimestamp(dataTime);\n                    if (dataTime < state.getRecordEarliestTimestamp()) {\n                        //更新最早时间戳，用于状态触发时候，作为session 窗口的begin时间戳\n                        state.setRecordEarliestTimestamp(dataTime);\n                    }\n\n                    //如果是最后一个窗口，更新窗口结束时间\n                    if (i == pairs.size() - 1) {\n                        long mayBeSessionEnd = dataTime + windowInfo.getSessionTimeout().toMilliseconds();\n                        if (windowKey.getWindowEnd() < mayBeSessionEnd) {\n                            logger.debug(\"update exist session window, before:[{} - {}], after:[{} - {}]\", Utils.format(windowKey.getWindowStart()), Utils.format(windowKey.getWindowEnd()),\n                                    Utils.format(windowKey.getWindowStart()), Utils.format(mayBeSessionEnd));\n                            //删除老状态\n                            needToDelete = windowKey;\n                            //需要保存的新状态\n                            windowKey = new WindowKey(windowKey.getOperatorName(), windowKey.getKey2String(), mayBeSessionEnd, windowKey.getWindowStart());\n                        }\n                    }\n                } else {\n                    logger.warn(\"discard data: key=[{}], data=[{}], dataTime=[{}], watermark=[{}]\", key, data, dataTime, watermark);\n                }\n\n                this.windowStore.put(stateTopicMessageQueue, windowKey, state);\n\n                this.idleWindowScaner.putAggregateSessionWindowCallback(windowKey, watermark, this.aggregateSessionWindowFire);\n                this.idleWindowScaner.removeOldAggregateSession(needToDelete);\n\n                this.windowStore.deleteByKey(needToDelete);\n            }\n\n            if (pairs.size() == 0 || createNewSessionWindow) {\n                return new Pair<>(lastStateSessionEnd, dataTime + windowInfo.getSessionTimeout().toMilliseconds());\n            }\n            return null;\n        }\n    }\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/metadata/Data.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.metadata;\n\nimport java.util.Properties;\n\npublic class Data<K, V> {\n    private Properties header;\n    private K key;\n    private V value;\n    private Long timestamp;\n\n    public Data(K key, V value, Long timestamp, Properties header) {\n        this.key = key;\n        this.value = value;\n        this.timestamp = timestamp;\n        this.header = header;\n    }\n\n    public K getKey() {\n        return key;\n    }\n\n    public void setKey(K key) {\n        this.key = key;\n    }\n\n    public V getValue() {\n        return value;\n    }\n\n    public void setValue(V value) {\n        this.value = value;\n    }\n\n    public Long getTimestamp() {\n        return timestamp;\n    }\n\n    public void setTimestamp(Long timestamp) {\n        this.timestamp = timestamp;\n    }\n\n    public Properties getHeader() {\n        return header;\n    }\n\n    public void setHeader(Properties header) {\n        this.header = header;\n    }\n\n    public <NK> Data<NK, V> key(NK key) {\n        return new Data<>(key, value, timestamp, new Properties(this.header));\n    }\n\n    public <NV> Data<K, NV> value(NV value) {\n        return new Data<>(key, value, timestamp, new Properties(this.header));\n    }\n\n    @Override\n    public String toString() {\n        return \"Data{\" +\n                \"key=\" + key +\n                \", value=\" + value +\n                \", timestamp=\" + timestamp +\n                '}';\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/metadata/StreamConfig.java",
    "content": "package org.apache.rocketmq.streams.core.metadata;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\npublic class StreamConfig {\n    public static final String TIME_TYPE = \"timeType\";\n    public static final String ALLOW_LATENESS_MILLISECOND = \"allowLatenessMillisecond\";\n    public static final String IDLE_TIME_TO_FIRE_WINDOW = \"idleTimeToFireWindow\";\n    public static final String ROCKETMQ_STREAMS_CONSUMER_GROUP = \"__source_shuffle_group\";\n    public static final String ROCKETMQ_STREAMS_STATE_CONSUMER_GROUP = \"__state_group\";\n    public static final String COMMIT_STATE_INTERNAL_MS = \"commitStateIntervalMillisecond\";\n\n    public static final String ROCKETMQ_STREAMS_CONSUMER_FORM_WHERE = \"consume_from_where\";\n\n    public static Integer STREAMS_PARALLEL_THREAD_NUM = 1;\n    public static Integer SHUFFLE_TOPIC_QUEUE_NUM = 8;\n    public static Integer SCHEDULED_THREAD_NUM = 2;\n    public static Integer DEFAULT_ALLOW_LATE_MILLISECONDS = 2000;\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/GroupedStream.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.rstream;\n\nimport org.apache.rocketmq.streams.core.function.accumulator.Accumulator;\nimport org.apache.rocketmq.streams.core.function.FilterAction;\nimport org.apache.rocketmq.streams.core.function.SelectAction;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.window.WindowInfo;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueSerializer;\n\nimport java.util.function.Supplier;\n\npublic interface GroupedStream<K, V> {\n\n    GroupedStream<K, Integer> count();\n\n    <OUT> GroupedStream<K, Integer> count(SelectAction<OUT, V> selectAction);\n\n\n    GroupedStream<K, V> min(SelectAction<? extends Number, V> selectAction);\n\n\n    GroupedStream<K, V> max(SelectAction<? extends Number, V> selectAction);\n\n\n    GroupedStream<K, ? extends Number> sum(SelectAction<? extends Number, V> selectAction);\n\n\n    GroupedStream<K, V> filter(FilterAction<V> predictor);\n\n    <OUT> GroupedStream<K, OUT> map(ValueMapperAction<V, OUT> valueMapperAction);\n\n    <VR> RStream<VR> flatMap(ValueMapperAction<V,? extends Iterable<? extends VR>> valueMapperAction);\n\n\n    <OUT> GroupedStream<K, OUT> aggregate(Accumulator<V, OUT> accumulator);\n\n    WindowStream<K, V> window(WindowInfo windowInfo);\n\n    GroupedStream<K, V> addGraphNode(String name, Supplier<Processor<V>> supplier);\n\n    RStream<V> toRStream();\n\n    void sink(String topicName, KeyValueSerializer<K, V> serializer);\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/GroupedStreamImpl.java",
    "content": "package org.apache.rocketmq.streams.core.rstream;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.function.AggregateAction;\nimport org.apache.rocketmq.streams.core.function.FilterAction;\nimport org.apache.rocketmq.streams.core.function.SelectAction;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.function.accumulator.Accumulator;\nimport org.apache.rocketmq.streams.core.function.supplier.AccumulatorSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.AddTagSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.AggregateSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.FilterSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.MultiValueChangeSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.SinkSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.SumAggregate;\nimport org.apache.rocketmq.streams.core.function.supplier.ValueChangeSupplier;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.window.WindowInfo;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueSerializer;\nimport org.apache.rocketmq.streams.core.topology.virtual.GraphNode;\nimport org.apache.rocketmq.streams.core.topology.virtual.ProcessorNode;\nimport org.apache.rocketmq.streams.core.topology.virtual.ShuffleProcessorNode;\nimport org.apache.rocketmq.streams.core.topology.virtual.SinkGraphNode;\nimport org.apache.rocketmq.streams.core.util.OperatorNameMaker;\n\nimport java.util.function.Supplier;\n\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FILTER_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.COUNT_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.ACCUMULATE_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FLAT_MAP_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MAP_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MAX_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MIN_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SINK_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SUM_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.WINDOW_PREFIX;\n\npublic class GroupedStreamImpl<K, V> implements GroupedStream<K, V> {\n    private final Pipeline pipeline;\n    private final GraphNode parent;\n\n    public GroupedStreamImpl(Pipeline pipeline, GraphNode parent) {\n        this.pipeline = pipeline;\n        this.parent = parent;\n    }\n\n    @Override\n    public GroupedStream<K, Integer> count() {\n        String name = OperatorNameMaker.makeName(COUNT_PREFIX, pipeline.getJobId());\n\n        Supplier<Processor<V>> supplier = new AggregateSupplier<>(name, parent.getName(), () -> 0, (K key, V value, Integer agg) -> agg + 1);\n\n        GraphNode graphNode;\n        if (this.parent.shuffleNode()) {\n            graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent);\n    }\n\n    @Override\n    public <OUT> GroupedStream<K, Integer> count(SelectAction<OUT, V> selectAction) {\n        String name = OperatorNameMaker.makeName(COUNT_PREFIX, pipeline.getJobId());\n\n        Supplier<Processor<V>> supplier = new AggregateSupplier<>(name, parent.getName(), () -> 0, (K key, V value, Integer agg) -> agg + 1);\n\n        GraphNode graphNode;\n        if (this.parent.shuffleNode()) {\n            graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent);\n    }\n\n    @Override\n    public GroupedStream<K, V> min(SelectAction<? extends Number, V> selectAction) {\n        String name = OperatorNameMaker.makeName(MIN_PREFIX, pipeline.getJobId());\n\n        Supplier<Processor<V>> supplier = new AggregateSupplier<>(name, parent.getName(), () -> null, (AggregateAction<K, V, V>) (key, value, accumulator) -> {\n            Number number = selectAction.select(value);\n            if (accumulator == null) {\n                return value;\n            } else {\n                Number storedMin = selectAction.select(accumulator);\n                double newValue = number.doubleValue();\n                double oldValue = storedMin.doubleValue();\n\n                if (newValue < oldValue) {\n                    return value;\n                } else {\n                    return accumulator;\n                }\n            }\n        });\n\n        GraphNode graphNode;\n        if (this.parent.shuffleNode()) {\n            graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent);\n    }\n\n    @Override\n    public GroupedStream<K, V> max(SelectAction<? extends Number, V> selectAction) {\n        String name = OperatorNameMaker.makeName(MAX_PREFIX, pipeline.getJobId());\n        Supplier<Processor<V>> supplier = new AggregateSupplier<>(name, parent.getName(), () -> null, (AggregateAction<K, V, V>) (key, value, accumulator) -> {\n            Number number = selectAction.select(value);\n            if (accumulator == null) {\n                return value;\n            } else {\n                Number storedMax = selectAction.select(accumulator);\n                double newValue = number.doubleValue();\n                double oldValue = storedMax.doubleValue();\n\n                if (newValue > oldValue) {\n                    return value;\n                } else {\n                    return accumulator;\n                }\n            }\n        });\n\n        GraphNode graphNode;\n        if (this.parent.shuffleNode()) {\n            graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent);\n    }\n\n    @Override\n    public GroupedStream<K, ? extends Number> sum(SelectAction<? extends Number, V> selectAction) {\n        String name = OperatorNameMaker.makeName(SUM_PREFIX, pipeline.getJobId());\n        Supplier<Processor<V>> supplier = new AggregateSupplier<>(name, parent.getName(), () -> null, new SumAggregate<>(selectAction));\n\n        GraphNode graphNode;\n        if (this.parent.shuffleNode()) {\n            graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent);\n    }\n\n    @Override\n    public GroupedStream<K, V> filter(FilterAction<V> predictor) {\n        String name = OperatorNameMaker.makeName(FILTER_PREFIX, pipeline.getJobId());\n\n        FilterSupplier<V> supplier = new FilterSupplier<>(predictor);\n        GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n\n        return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent);\n    }\n\n    @Override\n    public <OUT> GroupedStream<K, OUT> map(ValueMapperAction<V, OUT> mapperAction) {\n        String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId());\n\n        ValueChangeSupplier<V, OUT> supplier = new ValueChangeSupplier<>(mapperAction);\n        GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n\n        return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent);\n    }\n\n    @Override\n    public <VR> RStream<VR> flatMap(ValueMapperAction<V, ? extends Iterable<? extends VR>> valueMapperAction) {\n        String name = OperatorNameMaker.makeName(FLAT_MAP_PREFIX, pipeline.getJobId());\n\n        MultiValueChangeSupplier<V, VR> changeSupplier = new MultiValueChangeSupplier<>(valueMapperAction);\n        GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), changeSupplier);\n\n        return this.pipeline.addRStreamVirtualNode(graphNode, parent);\n    }\n\n    @Override\n    public <OUT> GroupedStream<K, OUT> aggregate(Accumulator<V, OUT> accumulator) {\n        String name = OperatorNameMaker.makeName(ACCUMULATE_PREFIX, pipeline.getJobId());\n        Supplier<Processor<V>> supplier = new AccumulatorSupplier<>(name, parent.getName(), value -> value, accumulator);\n\n        GraphNode graphNode;\n        if (this.parent.shuffleNode()) {\n            graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent);\n    }\n\n    @Override\n    public WindowStream<K, V> window(WindowInfo windowInfo) {\n        //需要在window里面shuffle\n        String name = OperatorNameMaker.makeName(WINDOW_PREFIX, pipeline.getJobId());\n\n        ProcessorNode<V> node;\n\n        if (!this.parent.shuffleNode()) {\n            node = new ProcessorNode<>(name, parent.getName(), new AddTagSupplier<>());\n        } else if (windowInfo.getJoinStream() != null) {\n            node = new ShuffleProcessorNode<>(name, parent.getName(), new AddTagSupplier<>(windowInfo::getJoinStream));\n        } else {\n            node = new ShuffleProcessorNode<>(name, parent.getName(), new AddTagSupplier<>());\n        }\n\n        return this.pipeline.addWindowStreamVirtualNode(node, parent, windowInfo);\n    }\n\n    @Override\n    public GroupedStream<K, V> addGraphNode(String name, Supplier<Processor<V>> supplier) {\n        GraphNode graphNode;\n        if (this.parent.shuffleNode()) {\n            graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addGroupedStreamVirtualNode(graphNode, parent);\n    }\n\n    @Override\n    public RStream<V> toRStream() {\n        return new RStreamImpl<>(this.pipeline, parent);\n    }\n\n    @Override\n    public void sink(String topicName, KeyValueSerializer<K, V> serializer) {\n        String name = OperatorNameMaker.makeName(SINK_PREFIX, pipeline.getJobId());\n\n        SinkSupplier<K, V> sinkSupplier = new SinkSupplier<>(topicName, serializer);\n        GraphNode sinkGraphNode = new SinkGraphNode<>(name, parent.getName(), topicName, sinkSupplier);\n\n        pipeline.addVirtualSink(sinkGraphNode, parent);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/JoinedStream.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.rstream;\n\nimport org.apache.rocketmq.streams.core.function.SelectAction;\nimport org.apache.rocketmq.streams.core.function.ValueJoinAction;\nimport org.apache.rocketmq.streams.core.function.supplier.AddTagSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.JoinAggregateSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.JoinWindowAggregateSupplier;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.window.JoinType;\nimport org.apache.rocketmq.streams.core.window.StreamType;\nimport org.apache.rocketmq.streams.core.window.WindowInfo;\nimport org.apache.rocketmq.streams.core.topology.virtual.GraphNode;\nimport org.apache.rocketmq.streams.core.topology.virtual.ProcessorNode;\nimport org.apache.rocketmq.streams.core.util.OperatorNameMaker;\n\nimport java.util.ArrayList;\nimport java.util.List;\nimport java.util.Objects;\nimport java.util.function.Supplier;\n\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.ADD_TAG;\n\npublic class JoinedStream<V1, V2> {\n    private RStream<V1> leftStream;\n    private RStream<V2> rightStream;\n    private JoinType joinType;\n\n    public JoinedStream(RStream<V1> leftStream, RStream<V2> rightStream, JoinType joinType) {\n        this.leftStream = leftStream;\n        this.rightStream = rightStream;\n        this.joinType = joinType;\n    }\n\n    public <K> Where<K> where(SelectAction<K, V1> rightSelectAction) {\n        return new Where<>(rightSelectAction);\n    }\n\n    public class Where<K> {\n        private SelectAction<K, V1> leftSelectAction;\n        private SelectAction<K, V2> rightSelectAction;\n\n        public Where(SelectAction<K, V1> leftSelectAction) {\n            this.leftSelectAction = leftSelectAction;\n        }\n\n\n        public Where<K> equalTo(SelectAction<K, V2> rightSelectAction) {\n            this.rightSelectAction = rightSelectAction;\n            return this;\n        }\n\n        public <OUT> RStream<OUT> apply(ValueJoinAction<V1, V2, OUT> joinAction) {\n            List<String> temp = new ArrayList<>();\n            Pipeline leftStreamPipeline = JoinedStream.this.leftStream.getPipeline();\n            String jobId = leftStreamPipeline.getJobId();\n\n            String name = OperatorNameMaker.makeName(OperatorNameMaker.JOIN_PREFIX, jobId);\n            Supplier<Processor<? super OUT>> supplier = new JoinAggregateSupplier<>(name, joinType, joinAction);\n            ProcessorNode<OUT> commChild = new ProcessorNode(name, temp, supplier);\n\n\n            {\n                GroupedStream<K, V1> leftGroupedStream = JoinedStream.this.leftStream.keyBy(leftSelectAction);\n                String addTagName = OperatorNameMaker.makeName(ADD_TAG, jobId);\n                leftGroupedStream.addGraphNode(addTagName, new AddTagSupplier<>(() -> StreamType.LEFT_STREAM));\n\n                GraphNode lastNode = leftStreamPipeline.getLastNode();\n                temp.add(lastNode.getName());\n                commChild.addParent(lastNode);\n            }\n\n            Pipeline rightStreamPipeline = JoinedStream.this.rightStream.getPipeline();\n            String rightJobId = rightStreamPipeline.getJobId();\n            if (!Objects.equals(jobId, rightJobId)) {\n                throw new IllegalStateException(\"left stream and right stream must have same jobId.\");\n            }\n\n            {\n                GroupedStream<K, V2> rightGroupedStream = JoinedStream.this.rightStream.keyBy(rightSelectAction);\n                String addTagName = OperatorNameMaker.makeName(ADD_TAG, jobId);\n                rightGroupedStream.addGraphNode(addTagName, new AddTagSupplier<>(()-> StreamType.RIGHT_STREAM));\n\n                GraphNode lastNode = rightStreamPipeline.getLastNode();\n                temp.add(lastNode.getName());\n                commChild.addParent(lastNode);\n\n                lastNode.addChild(commChild);\n            }\n            return new RStreamImpl<>(leftStreamPipeline, commChild);\n        }\n\n        public JoinWindow<K> window(WindowInfo windowInfo) {\n            return new JoinWindow<>(this.leftSelectAction, this.rightSelectAction, windowInfo);\n        }\n    }\n\n    public class JoinWindow<K> {\n        private SelectAction<K, V1> leftSelectAction;\n        private SelectAction<K, V2> rightSelectAction;\n        private WindowInfo windowInfo;\n\n        public JoinWindow(SelectAction<K, V1> leftSelectAction, SelectAction<K, V2> rightSelectAction, WindowInfo windowInfo) {\n            this.leftSelectAction = leftSelectAction;\n            this.rightSelectAction = rightSelectAction;\n            this.windowInfo = windowInfo;\n        }\n\n        public <OUT> RStream<OUT> apply(ValueJoinAction<V1, V2, OUT> joinAction) {\n            List<String> temp = new ArrayList<>();\n            WindowInfo.JoinStream joinStream = new WindowInfo.JoinStream(JoinedStream.this.joinType, null);\n            windowInfo.setJoinStream(joinStream);\n\n            Pipeline leftStreamPipeline = JoinedStream.this.leftStream.getPipeline();\n            String jobId = leftStreamPipeline.getJobId();\n\n            String name = OperatorNameMaker.makeName(OperatorNameMaker.JOIN_WINDOW_PREFIX, jobId);\n            Supplier<Processor<? super OUT>> supplier = new JoinWindowAggregateSupplier<>(name, windowInfo, joinAction);\n            ProcessorNode<OUT> commChild = new ProcessorNode(name, temp, supplier);\n\n\n            {\n                GroupedStream<K, V1> leftGroupedStream = JoinedStream.this.leftStream.keyBy(leftSelectAction);\n\n                WindowInfo leftWindowInfo = this.copy(windowInfo);\n\n                WindowInfo.JoinStream leftStream = new WindowInfo.JoinStream(JoinedStream.this.joinType, StreamType.LEFT_STREAM);\n                leftWindowInfo.setJoinStream(leftStream);\n\n                leftGroupedStream.window(leftWindowInfo);\n\n                GraphNode lastNode = leftStreamPipeline.getLastNode();\n                temp.add(lastNode.getName());\n                commChild.addParent(lastNode);\n            }\n\n            {\n\n                GroupedStream<K, V2> rightGroupedStream = JoinedStream.this.rightStream.keyBy(rightSelectAction);\n\n                WindowInfo rightWindowInfo = this.copy(windowInfo);\n\n                WindowInfo.JoinStream leftStream = new WindowInfo.JoinStream(JoinedStream.this.joinType, StreamType.RIGHT_STREAM);\n                rightWindowInfo.setJoinStream(leftStream);\n\n                rightGroupedStream.window(rightWindowInfo);\n\n                Pipeline rightStreamPipeline = JoinedStream.this.rightStream.getPipeline();\n                String rightJobId = rightStreamPipeline.getJobId();\n                if (!Objects.equals(jobId, rightJobId)) {\n                    throw new IllegalStateException(\"left stream and right stream must have same jobId.\");\n                }\n\n                GraphNode lastNode = rightStreamPipeline.getLastNode();\n                temp.add(lastNode.getName());\n                commChild.addParent(lastNode);\n\n                lastNode.addChild(commChild);\n            }\n\n            return new RStreamImpl<>(leftStreamPipeline, commChild);\n        }\n\n        private WindowInfo copy(WindowInfo windowInfo) {\n            WindowInfo result = new WindowInfo();\n\n            WindowInfo.JoinStream joinStream = windowInfo.getJoinStream();\n\n            if (joinStream != null) {\n                WindowInfo.JoinStream stream = new WindowInfo.JoinStream(joinStream.getJoinType(), joinStream.getStreamType());\n                result.setJoinStream(stream);\n            }\n\n            result.setSessionTimeout(windowInfo.getSessionTimeout());\n            result.setWindowType(windowInfo.getWindowType());\n            result.setWindowSize(windowInfo.getWindowSize());\n            result.setWindowSlide(windowInfo.getWindowSlide());\n\n            return result;\n        }\n\n\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/Pipeline.java",
    "content": "package org.apache.rocketmq.streams.core.rstream;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.window.WindowInfo;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.topology.virtual.AbstractGraphNode;\nimport org.apache.rocketmq.streams.core.topology.virtual.GraphNode;\n\nimport java.util.ArrayList;\nimport java.util.List;\n\npublic class Pipeline {\n    private final String jobId;\n    private final List<GraphNode> virtualNodes = new ArrayList<>();\n    private final GraphNode root = new AbstractGraphNode(\"root\") {\n        @Override\n        public void addRealNode(TopologyBuilder builder) {\n            //no-op\n        }\n    };\n\n    public Pipeline(String jobId) {\n        this.jobId = jobId;\n    }\n\n    public <T> RStream<T> addVirtualSource(GraphNode sourceGraphNode) {\n        root.addChild(sourceGraphNode);\n        virtualNodes.add(sourceGraphNode);\n\n        return new RStreamImpl<>(this, sourceGraphNode);\n    }\n\n    public <T> RStream<T> addRStreamVirtualNode(GraphNode currentNode, GraphNode parentNode) {\n        if (!virtualNodes.contains(parentNode)) {\n            virtualNodes.add(parentNode);\n        }\n\n        parentNode.addChild(currentNode);\n        currentNode.addParent(parentNode);\n\n        virtualNodes.add(currentNode);\n        return new RStreamImpl<>(this, currentNode);\n    }\n\n    public <K, V> GroupedStreamImpl<K, V> addGroupedStreamVirtualNode(GraphNode currentNode, GraphNode parentNode) {\n        if (!virtualNodes.contains(parentNode)) {\n            virtualNodes.add(parentNode);\n        }\n\n        parentNode.addChild(currentNode);\n        currentNode.addParent(parentNode);\n\n        virtualNodes.add(currentNode);\n        return new GroupedStreamImpl<>(this, currentNode);\n    }\n\n    public <K, V> WindowStream<K, V> addWindowStreamVirtualNode(GraphNode currentNode, GraphNode parentNode, WindowInfo windowInfo) {\n        if (!virtualNodes.contains(parentNode)) {\n            virtualNodes.add(parentNode);\n        }\n\n        parentNode.addChild(currentNode);\n        currentNode.addParent(parentNode);\n\n        virtualNodes.add(currentNode);\n        return new WindowStreamImpl<>(this, currentNode, windowInfo);\n    }\n\n    public void addVirtualSink(GraphNode currentNode, GraphNode parentNode) {\n        if (!virtualNodes.contains(parentNode)) {\n            virtualNodes.add(parentNode);\n        }\n        parentNode.addChild(currentNode);\n        virtualNodes.add(currentNode);\n    }\n\n    public GraphNode getRoot() {\n        return this.root;\n    }\n\n\n    public GraphNode getLastNode() {\n        return this.virtualNodes.get(virtualNodes.size() - 1);\n    }\n\n    public String getJobId() {\n        return jobId;\n    }\n\n    public int getVirtualNodesNum() {\n        return virtualNodes.size();\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/RStream.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.rstream;\n\nimport org.apache.rocketmq.streams.core.function.FilterAction;\nimport org.apache.rocketmq.streams.core.function.ForeachAction;\nimport org.apache.rocketmq.streams.core.function.SelectAction;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueSerializer;\n\npublic interface RStream<T> {\n    RStream<T> selectTimestamp(ValueMapperAction<T, Long> timestampSelector);\n\n    <O> RStream<O> map(ValueMapperAction<T, O> mapperAction);\n\n    <VR> RStream<VR> flatMap(final ValueMapperAction<T, ? extends Iterable<? extends VR>> mapper);\n\n    RStream<T> filter(FilterAction<T> predictor);\n\n    <K> GroupedStream<K, T> keyBy(SelectAction<K, T> selectAction);\n\n    void print();\n\n    RStream<T> foreach(ForeachAction<T> foreachAction);\n\n    <T2> JoinedStream<T, T2> join(RStream<T2> rightStream);\n\n    <T2> JoinedStream<T, T2> leftJoin(RStream<T2> rightStream);\n\n    Pipeline getPipeline();\n\n    void sink(String topicName, KeyValueSerializer<Object, T> serializer);\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/RStreamImpl.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.rstream;\n\n\nimport org.apache.rocketmq.streams.core.function.supplier.MultiValueChangeSupplier;\nimport org.apache.rocketmq.streams.core.util.OperatorNameMaker;\nimport org.apache.rocketmq.streams.core.function.FilterAction;\nimport org.apache.rocketmq.streams.core.function.ForeachAction;\nimport org.apache.rocketmq.streams.core.function.SelectAction;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.function.supplier.FilterSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.ForeachSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.KeySelectSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.PrintSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.SinkSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.TimestampSelectorSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.ValueChangeSupplier;\nimport org.apache.rocketmq.streams.core.window.JoinType;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueSerializer;\nimport org.apache.rocketmq.streams.core.topology.virtual.GraphNode;\nimport org.apache.rocketmq.streams.core.topology.virtual.ProcessorNode;\nimport org.apache.rocketmq.streams.core.topology.virtual.SinkGraphNode;\n\n\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FILTER_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FLAT_MAP_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FOR_EACH_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.GROUPBY_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MAP_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.PRINT_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SINK_PREFIX;\n\npublic class RStreamImpl<T> implements RStream<T> {\n    private final Pipeline pipeline;\n    private final GraphNode parent;\n\n    public RStreamImpl(Pipeline pipeline, GraphNode parent) {\n        this.pipeline = pipeline;\n        this.parent = parent;\n    }\n\n    @Override\n    public RStream<T> selectTimestamp(ValueMapperAction<T, Long> timestampSelector) {\n        String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId());\n\n        TimestampSelectorSupplier<T> supplier = new TimestampSelectorSupplier<>(timestampSelector);\n        GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), supplier);\n\n        return pipeline.addRStreamVirtualNode(processorNode, parent);\n    }\n\n    @Override\n    public <O> RStream<O> map(ValueMapperAction<T, O> mapperAction) {\n        String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId());\n\n        ValueChangeSupplier<T, O> supplier = new ValueChangeSupplier<>(mapperAction);\n        GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), supplier);\n\n        return pipeline.addRStreamVirtualNode(processorNode, parent);\n    }\n\n    @Override\n    public <VR> RStream<VR> flatMap(ValueMapperAction<T, ? extends Iterable<? extends VR>> mapper) {\n        String name = OperatorNameMaker.makeName(FLAT_MAP_PREFIX, pipeline.getJobId());\n\n        MultiValueChangeSupplier<T, VR> changeSupplier = new MultiValueChangeSupplier<>(mapper);\n        GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), changeSupplier);\n\n        return pipeline.addRStreamVirtualNode(processorNode, parent);\n    }\n\n    @Override\n    public RStream<T> filter(FilterAction<T> predictor) {\n        String name = OperatorNameMaker.makeName(FILTER_PREFIX, pipeline.getJobId());\n\n        FilterSupplier<T> supplier = new FilterSupplier<>(predictor);\n        GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), supplier);\n\n        return pipeline.addRStreamVirtualNode(processorNode, parent);\n    }\n\n    @Override\n    public <K> GroupedStream<K, T> keyBy(SelectAction<K, T> selectAction) {\n        String name = OperatorNameMaker.makeName(GROUPBY_PREFIX, pipeline.getJobId());\n\n        KeySelectSupplier<K, T> keySelectSupplier = new KeySelectSupplier<>(selectAction);\n\n        GraphNode processorNode = new ProcessorNode<>(name, parent.getName(), true, keySelectSupplier);\n\n        return pipeline.addGroupedStreamVirtualNode(processorNode, parent);\n    }\n\n    @Override\n    public void print() {\n        String name = OperatorNameMaker.makeName(PRINT_PREFIX, pipeline.getJobId());\n\n        PrintSupplier<T> printSupplier = new PrintSupplier<>();\n        GraphNode sinkGraphNode = new SinkGraphNode<>(name, parent.getName(), null, printSupplier);\n\n        pipeline.addVirtualSink(sinkGraphNode, parent);\n    }\n\n    @Override\n    public RStream<T> foreach(ForeachAction<T> foreachAction) {\n        String name = OperatorNameMaker.makeName(FOR_EACH_PREFIX, pipeline.getJobId());\n\n        ForeachSupplier<T> supplier = new ForeachSupplier<T>(foreachAction);\n\n        ProcessorNode<T> node = new ProcessorNode<>(name, parent.getName(), supplier);\n\n        return pipeline.addRStreamVirtualNode(node, parent);\n    }\n\n    @Override\n    public <T2> JoinedStream<T, T2> join(RStream<T2> rightStream) {\n        return new JoinedStream<>(this, rightStream, JoinType.INNER_JOIN);\n    }\n\n    @Override\n    public <T2> JoinedStream<T, T2> leftJoin(RStream<T2> rightStream) {\n        return new JoinedStream<>(this, rightStream, JoinType.LEFT_JOIN);\n    }\n\n    @Override\n    public Pipeline getPipeline() {\n        return pipeline;\n    }\n\n    @Override\n    public void sink(String topicName, KeyValueSerializer<Object, T> serializer) {\n        String name = OperatorNameMaker.makeName(SINK_PREFIX, pipeline.getJobId());\n\n        SinkSupplier<Object, T> sinkSupplier = new SinkSupplier<>(topicName, serializer);\n        GraphNode sinkGraphNode = new SinkGraphNode<>(name, parent.getName(), topicName, sinkSupplier);\n\n        pipeline.addVirtualSink(sinkGraphNode, parent);\n    }\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/StreamBuilder.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.rstream;\n\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueDeserializer;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.topology.virtual.GraphNode;\nimport org.apache.rocketmq.streams.core.topology.virtual.SourceGraphNode;\nimport org.apache.rocketmq.streams.core.util.OperatorNameMaker;\n\nimport java.util.ArrayList;\nimport java.util.List;\n\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SOURCE_PREFIX;\n\npublic class StreamBuilder {\n    private final List<Pipeline> pipelines = new ArrayList<>();\n    private final TopologyBuilder topologyBuilder;\n    private final String jobId;\n\n    public StreamBuilder(String jobId) throws Throwable {\n        checkJobId(jobId);\n        this.jobId = jobId;\n        this.topologyBuilder = new TopologyBuilder(jobId);\n    }\n\n    private void checkJobId(String jobId) {\n        if (jobId.startsWith(Constant.WATERMARK_KEY)) {\n            throw new IllegalArgumentException(\"jobId startsWith watermarkPrefix. jodId=\" + jobId);\n        }\n    }\n\n    public <OUT> RStream<OUT> source(String topicName, KeyValueDeserializer<Void, OUT> deserializer) {\n        Pipeline pipeline = new Pipeline(jobId);\n        this.pipelines.add(pipeline);\n\n        String name = OperatorNameMaker.makeName(SOURCE_PREFIX, jobId);\n\n        GraphNode sourceGraphNode = new SourceGraphNode<>(name, topicName, deserializer);\n\n        return pipeline.addVirtualSource(sourceGraphNode);\n    }\n\n    public TopologyBuilder build() {\n        //双流join场景中，添加共同节点的pipeline最后构建；三流join未验证。\n        pipelines.sort((o1, o2) -> o2.getVirtualNodesNum() - o1.getVirtualNodesNum());\n\n        for (Pipeline pipeline : pipelines) {\n            doBuild(pipeline.getRoot());\n        }\n        return topologyBuilder;\n    }\n\n    private void doBuild(GraphNode graphNode) {\n        graphNode.addRealNode(topologyBuilder);\n\n        List<GraphNode> allChild = graphNode.getAllChild();\n        for (GraphNode node : allChild) {\n            doBuild(node);\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/WindowStream.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.rstream;\n\nimport org.apache.rocketmq.streams.core.function.AggregateAction;\nimport org.apache.rocketmq.streams.core.function.FilterAction;\nimport org.apache.rocketmq.streams.core.function.SelectAction;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.function.accumulator.Accumulator;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueSerializer;\n\n/**\n * WindowStream must be generated from keyBy.\n * @param <K> key type\n * @param <V> value type\n */\npublic interface WindowStream<K, V> {\n    WindowStream<K, Integer> count();\n\n    WindowStream<K, Double> avg();\n\n    WindowStream<K, V> min(SelectAction<? extends Number, V> selectAction);\n\n    WindowStream<K, V> max(SelectAction<? extends Number, V> selectAction);\n\n    WindowStream<K, ? extends Number> sum(SelectAction<? extends Number, V> selectAction);\n\n    WindowStream<K, V> filter(FilterAction<V> predictor);\n\n    <OUT> WindowStream<K, OUT> map(ValueMapperAction<V, OUT> mapperAction);\n\n    <OUT> WindowStream<K, OUT> aggregate(AggregateAction<K, V, OUT> aggregateAction);\n\n    <OUT> WindowStream<K, OUT> aggregate(Accumulator<V, OUT> accumulator);\n\n    RStream<V> toRStream();\n\n    void sink(String topicName, KeyValueSerializer<K, V> serializer);\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/rstream/WindowStreamImpl.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.rstream;\n\nimport org.apache.rocketmq.streams.core.function.AggregateAction;\nimport org.apache.rocketmq.streams.core.function.FilterAction;\nimport org.apache.rocketmq.streams.core.function.SelectAction;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.function.accumulator.Accumulator;\nimport org.apache.rocketmq.streams.core.function.accumulator.AvgAccumulator;\nimport org.apache.rocketmq.streams.core.function.accumulator.CountAccumulator;\nimport org.apache.rocketmq.streams.core.function.supplier.AggregateSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.FilterSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.SinkSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.SumAggregate;\nimport org.apache.rocketmq.streams.core.function.supplier.ValueChangeSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.WindowAccumulatorSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.WindowAggregateSupplier;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueSerializer;\nimport org.apache.rocketmq.streams.core.topology.virtual.GraphNode;\nimport org.apache.rocketmq.streams.core.topology.virtual.ProcessorNode;\nimport org.apache.rocketmq.streams.core.topology.virtual.ShuffleProcessorNode;\nimport org.apache.rocketmq.streams.core.topology.virtual.SinkGraphNode;\nimport org.apache.rocketmq.streams.core.util.OperatorNameMaker;\nimport org.apache.rocketmq.streams.core.window.WindowInfo;\n\nimport java.util.function.Supplier;\n\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.COUNT_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.FILTER_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MAP_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MAX_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.MIN_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SINK_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SUM_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.WINDOW_AVG_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.AGGREGATE_PREFIX;\n\npublic class WindowStreamImpl<K, V> implements WindowStream<K, V> {\n    private final Pipeline pipeline;\n    private final GraphNode parent;\n    private final WindowInfo windowInfo;\n\n    public WindowStreamImpl(Pipeline pipeline, GraphNode parent, WindowInfo windowInfo) {\n        this.pipeline = pipeline;\n        this.parent = parent;\n        this.windowInfo = windowInfo;\n    }\n\n    @Override\n    public WindowStream<K, Integer> count() {\n        String name = OperatorNameMaker.makeName(COUNT_PREFIX, pipeline.getJobId());\n        Supplier<Processor<V>> supplier = new WindowAccumulatorSupplier<>(name, windowInfo, value -> value, new CountAccumulator<>());\n\n        //是否需要分组计算\n        ProcessorNode<V> node;\n        if (this.parent.shuffleNode()) {\n            node = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            node = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addWindowStreamVirtualNode(node, parent, windowInfo);\n    }\n\n    @Override\n    public WindowStream<K, Double> avg() {\n        String name = OperatorNameMaker.makeName(WINDOW_AVG_PREFIX, pipeline.getJobId());\n        Supplier<Processor<V>> supplier = new WindowAccumulatorSupplier<>(name, windowInfo, value -> value, new AvgAccumulator<>());\n\n        //是否需要分组计算\n        ProcessorNode<V> node;\n        if (this.parent.shuffleNode()) {\n            node = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            node = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addWindowStreamVirtualNode(node, parent, windowInfo);\n    }\n\n    @Override public WindowStream<K, V> min(SelectAction<? extends Number, V> selectAction) {\n        String name = OperatorNameMaker.makeName(MIN_PREFIX, pipeline.getJobId());\n\n        Supplier<Processor<V>> supplier = new WindowAggregateSupplier<>(name, windowInfo, () -> null, (AggregateAction<K, V, V>) (key, value, accumulator) -> {\n            Number number = selectAction.select(value);\n            if (accumulator == null) {\n                return value;\n            } else {\n                Number storedMin = selectAction.select(accumulator);\n                double newValue = number.doubleValue();\n                double oldValue = storedMin.doubleValue();\n\n                if (newValue < oldValue) {\n                    return value;\n                } else {\n                    return accumulator;\n                }\n            }\n        });\n\n        GraphNode graphNode;\n        if (this.parent.shuffleNode()) {\n            graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo);\n    }\n\n    @Override public WindowStream<K, V> max(SelectAction<? extends Number, V> selectAction) {\n        String name = OperatorNameMaker.makeName(MAX_PREFIX, pipeline.getJobId());\n\n        Supplier<Processor<V>> supplier = new WindowAggregateSupplier<>(name, windowInfo, () -> null, (AggregateAction<K, V, V>) (key, value, accumulator) -> {\n            Number number = selectAction.select(value);\n            if (accumulator == null) {\n                return value;\n            } else {\n                Number storedMax = selectAction.select(accumulator);\n                double newValue = number.doubleValue();\n                double oldValue = storedMax.doubleValue();\n\n                if (newValue > oldValue) {\n                    return value;\n                } else {\n                    return accumulator;\n                }\n            }\n        });\n\n        GraphNode graphNode;\n        if (this.parent.shuffleNode()) {\n            graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo);\n    }\n\n    @Override public WindowStream<K, ? extends Number> sum(SelectAction<? extends Number, V> selectAction) {\n        String name = OperatorNameMaker.makeName(SUM_PREFIX, pipeline.getJobId());\n        Supplier<Processor<V>> supplier = new WindowAggregateSupplier<>(name, windowInfo, () -> null, new SumAggregate<>(selectAction));\n\n        GraphNode graphNode;\n        if (this.parent.shuffleNode()) {\n            graphNode = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo);\n    }\n\n    @Override\n    public WindowStream<K, V> filter(FilterAction<V> predictor) {\n        String name = OperatorNameMaker.makeName(FILTER_PREFIX, pipeline.getJobId());\n\n        FilterSupplier<V> supplier = new FilterSupplier<>(predictor);\n        GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n\n        return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo);\n    }\n\n    @Override\n    public <OUT> WindowStream<K, OUT> map(ValueMapperAction<V, OUT> mapperAction) {\n        String name = OperatorNameMaker.makeName(MAP_PREFIX, pipeline.getJobId());\n\n        ValueChangeSupplier<V, OUT> supplier = new ValueChangeSupplier<>(mapperAction);\n        GraphNode graphNode = new ProcessorNode<>(name, parent.getName(), supplier);\n\n        return this.pipeline.addWindowStreamVirtualNode(graphNode, parent, windowInfo);\n    }\n\n    @Override\n    public <OUT> WindowStream<K, OUT> aggregate(AggregateAction<K, V, OUT> aggregateAction) {\n        String name = OperatorNameMaker.makeName(AGGREGATE_PREFIX, pipeline.getJobId());\n\n        Supplier<Processor<V>> supplier = new WindowAggregateSupplier<>(name, windowInfo, () -> null, aggregateAction);\n\n        //是否需要分组计算\n        ProcessorNode<V> node;\n\n        if (this.parent.shuffleNode()) {\n            node = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            node = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addWindowStreamVirtualNode(node, parent, windowInfo);\n    }\n\n    @Override\n    public <OUT> WindowStream<K, OUT> aggregate(Accumulator<V, OUT> accumulator) {\n        String name = OperatorNameMaker.makeName(AGGREGATE_PREFIX, pipeline.getJobId());\n\n        Supplier<Processor<V>> supplier = new WindowAccumulatorSupplier<>(name, windowInfo, value -> value, accumulator);\n\n        //是否需要分组计算\n        ProcessorNode<V> node;\n\n        if (this.parent.shuffleNode()) {\n            node = new ShuffleProcessorNode<>(name, parent.getName(), supplier);\n        } else {\n            node = new ProcessorNode<>(name, parent.getName(), supplier);\n        }\n\n        return this.pipeline.addWindowStreamVirtualNode(node, parent, windowInfo);\n    }\n\n    @Override\n    public RStream<V> toRStream() {\n        return new RStreamImpl<>(this.pipeline, parent);\n    }\n\n    @Override\n    public void sink(String topicName, KeyValueSerializer<K, V> serializer) {\n        String name = OperatorNameMaker.makeName(SINK_PREFIX, pipeline.getJobId());\n\n        SinkSupplier<K, V> sinkSupplier = new SinkSupplier<>(topicName, serializer);\n        GraphNode sinkGraphNode = new SinkGraphNode<>(name, parent.getName(), topicName, sinkSupplier);\n\n        pipeline.addVirtualSink(sinkGraphNode, parent);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/AbstractProcessor.java",
    "content": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n\nimport com.fasterxml.jackson.core.JsonProcessingException;\nimport io.netty.buffer.ByteBuf;\nimport io.netty.buffer.ByteBufAllocator;\nimport io.netty.buffer.Unpooled;\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.metadata.StreamConfig;\nimport org.apache.rocketmq.streams.core.state.StateStore;\nimport org.apache.rocketmq.streams.core.util.Utils;\n\nimport java.io.IOException;\nimport java.nio.charset.StandardCharsets;\nimport java.util.ArrayList;\nimport java.util.Collections;\nimport java.util.List;\n\npublic abstract class AbstractProcessor<T> implements Processor<T> {\n    private final List<Processor<T>> children = new ArrayList<>();\n    protected StreamContext<T> context;\n    protected long allowDelay = 0;\n\n    @Override\n    public void addChild(Processor<T> processor) {\n        children.add(processor);\n    }\n\n    @Override\n    public void preProcess(StreamContext<T> context) throws RecoverStateStoreThrowable {\n        this.context = context;\n        this.context.init(getChildren());\n        Object delayObj = this.context\n                .getUserProperties()\n                .getOrDefault(StreamConfig.ALLOW_LATENESS_MILLISECOND, StreamConfig.DEFAULT_ALLOW_LATE_MILLISECONDS);\n\n        this.allowDelay = Long.parseLong(String.valueOf(delayObj));\n    }\n\n    protected List<Processor<T>> getChildren() {\n        return Collections.unmodifiableList(children);\n    }\n\n    protected StateStore waitStateReplay() throws RecoverStateStoreThrowable {\n        MessageQueue sourceTopicQueue = new MessageQueue(context.getSourceTopic(), context.getSourceBrokerName(), context.getSourceQueueId());\n\n        StateStore stateStore = context.getStateStore();\n        stateStore.waitIfNotReady(sourceTopicQueue);\n        return stateStore;\n    }\n\n    @SuppressWarnings(\"unchecked\")\n    protected <KEY> Data<KEY, T> convert(Data<?, ?> data) {\n        return (Data<KEY, T>) new Data<>(data.getKey(), data.getValue(), data.getTimestamp(), data.getHeader());\n    }\n\n\n\n    private final ByteBuf buf = Unpooled.buffer(16);\n    /**\n     * encode\n     * <pre>\n     * +-----------+---------------+-------------+-------------+\n     * | Int(4)    |   className  | Int(4)       | value bytes |\n     * | classname |              |object length |             |\n     * +-----------+--------------+---------------+-------------+\n     * </pre>\n     * @param obj the object to serialize;\n     * @return byte[] the result of serialize\n     * @throws JsonProcessingException serialize exception.\n     */\n    protected byte[] object2Byte(Object obj) throws JsonProcessingException {\n        if (obj == null) {\n            return new byte[]{};\n        }\n\n        String name = obj.getClass().getName();\n        byte[] className = name.getBytes(StandardCharsets.UTF_8);\n        byte[] objBytes = Utils.object2Byte(obj);\n\n\n        buf.writeInt(className.length);\n        buf.writeBytes(className);\n        buf.writeInt(objBytes.length);\n        buf.writeBytes(objBytes);\n\n\n        byte[] bytes = new byte[buf.readableBytes()];\n        buf.readBytes(bytes);\n\n        buf.clear();\n        return bytes;\n    }\n\n    /**\n     * decode\n     * <pre>\n     * +-----------+---------------+-------------+-------------+\n     * | Int(4)    |   className  | Int(4)       | value bytes |\n     * | classname |              |object length |             |\n     * +-----------+--------------+---------------+-------------+\n     * </pre>\n     * @param bytes the byte array to deserialize;\n     * @return V the result of deserialize\n     * @throws Throwable deserialize exception.\n     */\n    @SuppressWarnings(\"unchecked\")\n    public <V> V byte2Object(byte[] bytes) throws Throwable {\n        if (bytes == null || bytes.length == 0) {\n            return null;\n        }\n\n        ByteBuf byteBuf = Unpooled.wrappedBuffer(bytes);\n\n        int classNameLength = byteBuf.readInt();\n        ByteBuf classNameBuf = byteBuf.readBytes(classNameLength);\n\n        byte[] clazzNameBytes = new byte[classNameBuf.readableBytes()];\n        classNameBuf.readBytes(clazzNameBytes);\n        //实例化\n        String className = new String(clazzNameBytes, StandardCharsets.UTF_8);\n        Class<V> clazz = (Class<V>)Class.forName(className);\n\n        int objectLength = byteBuf.readInt();\n        ByteBuf objBuf = byteBuf.readBytes(objectLength);\n        byte[] objectBytes = new byte[objectLength];\n        objBuf.readBytes(objectBytes);\n\n        classNameBuf.release();\n        objBuf.release();\n\n        return Utils.byte2Object(objectBytes, clazz);\n    }\n\n    protected String toHexString(Object source) throws JsonProcessingException {\n        if (source == null) {\n            return null;\n        }\n        if (source instanceof String) {\n            return (String) source;\n        }\n        byte[] sourceByte = this.object2Byte(source);\n\n        return Utils.toHexString(sourceByte);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/AbstractWindowProcessor.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.running;\n\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RStreamsException;\nimport org.apache.rocketmq.streams.core.state.StateStore;\nimport org.apache.rocketmq.streams.core.util.ColumnFamilyUtil;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.streams.core.window.fire.IdleWindowScaner;\nimport org.apache.rocketmq.streams.core.window.Window;\nimport org.apache.rocketmq.streams.core.window.WindowInfo;\nimport org.apache.rocketmq.streams.core.window.fire.AccumulatorWindowFire;\nimport org.apache.rocketmq.streams.core.window.fire.AccumulatorSessionWindowFire;\nimport org.apache.rocketmq.streams.core.window.fire.AggregateSessionWindowFire;\nimport org.apache.rocketmq.streams.core.window.fire.AggregateWindowFire;\nimport org.apache.rocketmq.streams.core.window.fire.JoinWindowFire;\n\nimport java.util.ArrayList;\nimport java.util.List;\n\npublic abstract class AbstractWindowProcessor<V> extends AbstractProcessor<V> {\n    protected IdleWindowScaner idleWindowScaner;\n    protected AccumulatorWindowFire<?, ?, ?, ?> accumulatorWindowFire;\n    protected AccumulatorSessionWindowFire<?, ?, ?, ?> accumulatorSessionWindowFire;\n\n    protected AggregateWindowFire<?, ?, ?> aggregateWindowFire;\n    protected AggregateSessionWindowFire<?, ?, ?> aggregateSessionWindowFire;\n\n    protected JoinWindowFire<?, ?, ?, ?> joinWindowFire;\n\n    protected List<Window> calculateWindow(WindowInfo windowInfo, long valueTime) {\n        long sizeInterval = windowInfo.getWindowSize().toMillSecond();\n        long slideInterval = windowInfo.getWindowSlide().toMillSecond();\n\n        List<Window> result = new ArrayList<>((int) (sizeInterval / slideInterval));\n        long lastStart = valueTime - (valueTime + slideInterval) % slideInterval;\n\n        for (long start = lastStart; start > valueTime - sizeInterval; start -= slideInterval) {\n            long end = start + sizeInterval;\n            Window window = new Window(start, end);\n            result.add(window);\n        }\n        return result;\n    }\n\n\n    protected long watermark(long watermark, MessageQueue stateTopicMessageQueue) {\n        byte[] keyBytes = Utils.watermarkKeyBytes(stateTopicMessageQueue, Constant.WATERMARK_KEY);\n\n        try {\n            StateStore stateStore = this.context.getStateStore();\n\n            byte[] watermarkBytes = stateStore.get(ColumnFamilyUtil.WATERMARK_STATE_CF, keyBytes);\n            long oldWatermark = Utils.bytes2Long(watermarkBytes);\n\n            if (watermark > oldWatermark) {\n                byte[] newWatermarkBytes = Utils.long2Bytes(watermark);\n                stateStore.put(stateTopicMessageQueue, ColumnFamilyUtil.WATERMARK_STATE_CF, keyBytes, newWatermarkBytes);\n            } else {\n                watermark = oldWatermark;\n            }\n        } catch (Throwable t) {\n            throw new RStreamsException(t);\n        }\n\n\n        return watermark;\n    }\n\n\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/MessageQueueListenerWrapper.java",
    "content": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.client.consumer.MessageQueueListener;\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.HashSet;\nimport java.util.Set;\nimport java.util.concurrent.ConcurrentHashMap;\nimport java.util.function.BiFunction;\nimport java.util.function.Function;\n\nimport static org.apache.rocketmq.streams.core.common.Constant.STATE_TOPIC_SUFFIX;\n\nclass MessageQueueListenerWrapper implements MessageQueueListener {\n    private static final Logger logger = LoggerFactory.getLogger(MessageQueueListenerWrapper.class.getName());\n    private final MessageQueueListener originListener;\n    private final TopologyBuilder topologyBuilder;\n    private final ConcurrentHashMap<String, Set<MessageQueue>> ownedMapping = new ConcurrentHashMap<>();\n    private final ConcurrentHashMap<String, Processor<?>> mq2Processor = new ConcurrentHashMap<>();\n\n    private BiFunction<Set<MessageQueue>, Set<MessageQueue>, Throwable> recoverHandler;\n\n    private Function<Set<MessageQueue>, Throwable> resetOffsetHandler;\n\n    MessageQueueListenerWrapper(MessageQueueListener originListener, TopologyBuilder topologyBuilder) {\n        this.originListener = originListener;\n        this.topologyBuilder = topologyBuilder;\n    }\n\n    @Override\n    public void messageQueueChanged(String topic, Set<MessageQueue> mqAll, Set<MessageQueue> mqDivided) {\n        Set<MessageQueue> ownedQueues = ownedMapping.computeIfAbsent(topic, s -> new HashSet<>());\n\n        HashSet<MessageQueue> addQueue = new HashSet<>(mqDivided);\n        addQueue.removeAll(ownedQueues);\n\n        HashSet<MessageQueue> removeQueue = new HashSet<>(ownedQueues);\n        removeQueue.removeAll(mqDivided);\n\n        ownedQueues.addAll(new HashSet<>(addQueue));\n        ownedQueues.removeAll(new HashSet<>(removeQueue));\n\n        //从shuffle topic中读出的数据才能进行有状态计算。\n        if (topic.endsWith(Constant.SHUFFLE_TOPIC_SUFFIX)) {\n            Throwable throwable = this.recoverHandler.apply(addQueue, removeQueue);\n            if (throwable != null) {\n                throw new RuntimeException(throwable);\n            }\n            logger.info(\"recover messageQueue finish, addQueue: [{}], removeQueue:[{}].\", addQueue, removeQueue);\n        }\n\n        buildTask(addQueue);\n        //设计的不太好，移除q，添加消费任务之前，应该加一个状态移除函数;目前这样写的问题是：状态提前移除/加载了，consumer其实仍然在从某个将要移除的q中拉取数据，但是状态却被移除了。\n        //也不能把originListener.messageQueueChanged放在loadState/removeState之前，那样会已经在拉取数据了，但是状态没有加载好。\n        originListener.messageQueueChanged(topic, mqAll, mqDivided);\n        removeTask(removeQueue);\n    }\n\n\n    private void buildTask(Set<MessageQueue> addQueues) {\n        for (MessageQueue messageQueue : addQueues) {\n            String key = Utils.buildKey(messageQueue.getBrokerName(), messageQueue.getTopic(), messageQueue.getQueueId());\n            if (!mq2Processor.containsKey(key)) {\n                Processor<?> processor = topologyBuilder.build(messageQueue.getTopic());\n                this.mq2Processor.put(key, processor);\n            }\n        }\n    }\n\n    private void removeTask(Set<MessageQueue> removeQueues) {\n        for (MessageQueue removeQueue : removeQueues) {\n            String key = Utils.buildKey(removeQueue.getBrokerName(), removeQueue.getTopic(), removeQueue.getQueueId());\n            mq2Processor.remove(key);\n        }\n    }\n\n    @SuppressWarnings(\"unchecked\")\n    <T> Processor<T> selectProcessor(String key) {\n        return (Processor<T>) this.mq2Processor.get(key);\n    }\n\n    public void setRecoverHandler(BiFunction<Set<MessageQueue>, Set<MessageQueue>, Throwable> handler) {\n        this.recoverHandler = handler;\n    }\n\n    public void setResetOffsetHandler(Function<Set<MessageQueue>, Throwable> handler) {\n        this.resetOffsetHandler = handler;\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/Processor.java",
    "content": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable;\n\npublic interface Processor<T> {\n    void addChild(Processor<T> processor);\n\n\n    void preProcess(StreamContext<T> context) throws RecoverStateStoreThrowable;\n\n    void process(T data) throws Throwable;\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/RocketMQClient.java",
    "content": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;\nimport org.apache.rocketmq.client.exception.MQClientException;\nimport org.apache.rocketmq.client.producer.DefaultMQProducer;\nimport org.apache.rocketmq.common.consumer.ConsumeFromWhere;\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.common.protocol.route.QueueData;\nimport org.apache.rocketmq.common.protocol.route.TopicRouteData;\nimport org.apache.rocketmq.remoting.exception.RemotingException;\nimport org.apache.rocketmq.tools.admin.DefaultMQAdminExt;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.Collection;\nimport java.util.List;\nimport java.util.Set;\nimport java.util.UUID;\n\nimport static org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData.SUB_ALL;\nimport static org.apache.rocketmq.streams.core.common.Constant.*;\n\npublic class RocketMQClient {\n    private static final Logger logger = LoggerFactory.getLogger(RocketMQClient.class);\n    private final String nameSrvAddr;\n\n    public RocketMQClient(String nameSrvAddr) {\n        this.nameSrvAddr = nameSrvAddr;\n    }\n\n    public DefaultLitePullConsumer pullConsumer(String groupName,\n                                                Set<String> topics) throws MQClientException {\n        DefaultLitePullConsumer pullConsumer = new DefaultLitePullConsumer(groupName);\n        pullConsumer.setNamesrvAddr(nameSrvAddr);\n        pullConsumer.setAutoCommit(false);\n        pullConsumer.setPullBatchSize(1000);\n\n        for (String topic : topics) {\n            pullConsumer.subscribe(topic, SUB_ALL);\n            logger.debug(\"subscribe topic:{}, groupName:{}\", topic, groupName);\n        }\n\n        return pullConsumer;\n    }\n\n    public DefaultMQProducer producer(String groupName) {\n        DefaultMQProducer producer = new DefaultMQProducer(groupName);\n        producer.setNamesrvAddr(nameSrvAddr);\n        return producer;\n    }\n\n    public DefaultMQAdminExt getMQAdmin() throws MQClientException {\n        DefaultMQAdminExt mqAdminExt = new DefaultMQAdminExt(1000);\n        mqAdminExt.setInstanceName(UUID.randomUUID().toString());\n        mqAdminExt.setNamesrvAddr(nameSrvAddr);\n        mqAdminExt.start();\n        return mqAdminExt;\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/StreamContext.java",
    "content": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.client.producer.DefaultMQProducer;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.state.StateStore;\nimport org.apache.rocketmq.streams.core.window.fire.IdleWindowScaner;\n\nimport java.util.List;\nimport java.util.Properties;\n\npublic interface StreamContext<V> {\n    void init(List<Processor<V>> childrenProcessors);\n\n    StateStore getStateStore();\n\n    long getDataTime();\n\n    <K> K getKey();\n\n    Properties getUserProperties();\n\n    Properties getHeader();\n\n    DefaultMQProducer getDefaultMQProducer();\n\n    String getSourceBrokerName();\n\n    String getSourceTopic();\n\n    Integer getSourceQueueId();\n\n    IdleWindowScaner getDefaultWindowScaner();\n\n    StreamContext<V> copy();\n\n    <K> void forward(Data<K, V> data) throws Throwable;\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/StreamContextImpl.java",
    "content": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.client.producer.DefaultMQProducer;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.state.StateStore;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.streams.core.window.fire.IdleWindowScaner;\nimport org.apache.rocketmq.tools.admin.DefaultMQAdminExt;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.List;\nimport java.util.Properties;\n\n/**\n * 1、可以获得当前processor；\n * 2、可以获得下一个执行节点\n * 3、可获得动态的运行时信息，例如正在处理的数据来自那个topic，MQ，偏移量多少；\n */\npublic class StreamContextImpl<V> implements StreamContext<V> {\n    private static final Logger logger = LoggerFactory.getLogger(StreamContextImpl.class);\n\n    private final Properties properties;\n    private final DefaultMQProducer producer;\n    private final DefaultMQAdminExt mqAdmin;\n    private final StateStore stateStore;\n    private final String messageFromWhichSourceTopicQueue;\n    private final IdleWindowScaner idleWindowScaner;\n\n    private Object key;\n    private long dataTime;\n    private Properties header = new Properties();\n\n    private final List<Processor<V>> childList = new ArrayList<>();\n\n    StreamContextImpl(Properties properties,\n                      DefaultMQProducer producer,\n                      DefaultMQAdminExt mqAdmin,\n                      StateStore stateStore,\n                      String messageFromWhichSourceTopicQueue,\n                      IdleWindowScaner idleWindowScaner) {\n        this.properties = properties;\n        this.producer = producer;\n        this.mqAdmin = mqAdmin;\n        this.stateStore = stateStore;\n        this.messageFromWhichSourceTopicQueue = messageFromWhichSourceTopicQueue;\n        this.idleWindowScaner = idleWindowScaner;\n    }\n\n    @Override\n    public void init(List<Processor<V>> childrenProcessors) {\n        this.childList.clear();\n        if (childrenProcessors != null) {\n            this.childList.addAll(childrenProcessors);\n        }\n    }\n\n    @Override\n    public StateStore getStateStore() {\n        return this.stateStore;\n    }\n\n    @Override\n    public DefaultMQProducer getDefaultMQProducer() {\n        return producer;\n    }\n\n\n    public String getSourceBrokerName() {\n        String[] split = Utils.split(messageFromWhichSourceTopicQueue);\n        return split[0];\n    }\n\n    public String getSourceTopic() {\n        String[] split = Utils.split(messageFromWhichSourceTopicQueue);\n        return split[1];\n    }\n\n    public Integer getSourceQueueId() {\n        String[] split = Utils.split(messageFromWhichSourceTopicQueue);\n        return Integer.parseInt(split[2]);\n    }\n\n    @Override\n    public long getDataTime() {\n        return this.dataTime;\n    }\n\n\n    @Override\n    @SuppressWarnings(\"unchecked\")\n    public <K> K getKey() {\n        return (K) key;\n    }\n\n    <K> void setKey(K key) {\n        this.key = key;\n    }\n\n\n    @Override\n    public Properties getUserProperties() {\n        Properties result = new Properties();\n        result.putAll(this.properties);\n\n        return result;\n    }\n\n\n    @Override\n    public Properties getHeader() {\n        Properties result = new Properties();\n        result.putAll(this.header);\n\n        return result;\n    }\n\n    @Override\n    public IdleWindowScaner getDefaultWindowScaner() {\n        return this.idleWindowScaner;\n    }\n\n\n    @Override\n    public StreamContext<V> copy() {\n        StreamContextImpl<V> streamContext = new StreamContextImpl<>(this.properties,\n                this.producer,\n                this.mqAdmin,\n                this.stateStore,\n                this.messageFromWhichSourceTopicQueue,\n                this.idleWindowScaner);\n        streamContext.key = this.key;\n        streamContext.dataTime = this.dataTime;\n        streamContext.header = new Properties(this.header);\n        streamContext.childList.addAll(this.childList);\n\n        return streamContext;\n    }\n\n    @Override\n    public <K> void forward(Data<K, V> data) throws Throwable {\n        this.key = data.getKey();\n\n        if (data.getTimestamp() != null) {\n            this.dataTime = data.getTimestamp();\n        }\n\n        this.header = data.getHeader();\n\n        List<Processor<V>> store = new ArrayList<>(childList);\n\n        for (Processor<V> processor : childList) {\n\n            try {\n                processor.preProcess(this);\n                processor.process(data.getValue());\n            } finally {\n                this.childList.clear();\n                this.childList.addAll(store);\n            }\n        }\n    }\n\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/running/WorkerThread.java",
    "content": "package org.apache.rocketmq.streams.core.running;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.commons.lang3.StringUtils;\nimport org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;\nimport org.apache.rocketmq.client.consumer.MessageQueueListener;\nimport org.apache.rocketmq.client.exception.MQClientException;\nimport org.apache.rocketmq.client.producer.DefaultMQProducer;\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.common.admin.ConsumeStats;\nimport org.apache.rocketmq.common.admin.OffsetWrapper;\nimport org.apache.rocketmq.common.consumer.ConsumeFromWhere;\nimport org.apache.rocketmq.common.message.MessageExt;\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.common.protocol.body.ClusterInfo;\nimport org.apache.rocketmq.common.protocol.route.BrokerData;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.DataProcessThrowable;\nimport org.apache.rocketmq.streams.core.exception.RStreamsException;\nimport org.apache.rocketmq.streams.core.function.supplier.SourceSupplier;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.metadata.StreamConfig;\nimport org.apache.rocketmq.streams.core.window.fire.IdleWindowScaner;\nimport org.apache.rocketmq.streams.core.window.TimeType;\nimport org.apache.rocketmq.streams.core.state.RocketMQStore;\nimport org.apache.rocketmq.streams.core.state.RocksDBStore;\nimport org.apache.rocketmq.streams.core.state.StateStore;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.RocketMQUtil;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.tools.admin.DefaultMQAdminExt;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.HashSet;\nimport java.util.List;\nimport java.util.Map;\nimport java.util.Properties;\nimport java.util.Set;\nimport java.util.concurrent.ScheduledExecutorService;\nimport java.util.concurrent.TimeUnit;\n\nimport static org.apache.rocketmq.streams.core.common.Constant.*;\nimport static org.apache.rocketmq.streams.core.metadata.StreamConfig.ROCKETMQ_STREAMS_CONSUMER_FORM_WHERE;\nimport static org.apache.rocketmq.streams.core.metadata.StreamConfig.ROCKETMQ_STREAMS_CONSUMER_GROUP;\n\npublic class WorkerThread extends Thread {\n    private static final Logger logger = LoggerFactory.getLogger(WorkerThread.class.getName());\n    private final TopologyBuilder topologyBuilder;\n    private final PlanetaryEngine<?, ?> planetaryEngine;\n    private final Properties properties;\n    private final String jobId;\n    private final ScheduledExecutorService executor;\n\n\n    public WorkerThread(String threadName,\n                        TopologyBuilder topologyBuilder,\n                        Properties properties,\n                        ScheduledExecutorService executor) throws MQClientException {\n        super(threadName);\n\n        this.topologyBuilder = topologyBuilder;\n        this.properties = properties;\n        jobId = topologyBuilder.getJobId();\n        this.executor = executor;\n\n        String groupName = String.join(\"_\", jobId, ROCKETMQ_STREAMS_CONSUMER_GROUP);\n\n        RocketMQClient rocketMQClient = new RocketMQClient(properties.getProperty(MixAll.NAMESRV_ADDR_PROPERTY));\n\n        Set<String> topicNames = topologyBuilder.getSourceTopic();\n\n        DefaultLitePullConsumer unionConsumer = rocketMQClient.pullConsumer(groupName, topicNames);\n\n        MessageQueueListener originListener = unionConsumer.getMessageQueueListener();\n        MessageQueueListenerWrapper wrapper = new MessageQueueListenerWrapper(originListener, topologyBuilder);\n        unionConsumer.setMessageQueueListener(wrapper);\n\n        DefaultMQProducer producer = rocketMQClient.producer(groupName);\n        DefaultMQAdminExt mqAdmin = rocketMQClient.getMQAdmin();\n\n        RocksDBStore rocksDBStore = new RocksDBStore(threadName);\n        RocketMQStore store = new RocketMQStore(producer, rocksDBStore, mqAdmin, this.properties);\n\n        this.planetaryEngine = new PlanetaryEngine<>(unionConsumer, producer, store, mqAdmin, wrapper, topicNames);\n    }\n\n    @Override\n    public void run() {\n        try {\n            this.planetaryEngine.start();\n            logger.info(\"worker thread=[{}], start task success, jobId:{}\", this.getName(), jobId);\n\n            this.planetaryEngine.maybeResetOffsetToFirst();\n            this.planetaryEngine.runInLoop();\n        } catch (Throwable e) {\n            logger.error(\"worker thread=[{}], error:{}.\", this.getName(), e);\n            throw new RStreamsException(e);\n        } finally {\n            this.planetaryEngine.stop();\n        }\n    }\n\n    public void shutdown() {\n        this.planetaryEngine.stop();\n    }\n\n\n    @SuppressWarnings(\"unchecked\")\n    class PlanetaryEngine<K, V> {\n        private final DefaultLitePullConsumer unionConsumer;\n        private final DefaultMQProducer producer;\n        private final DefaultMQAdminExt mqAdmin;\n        private final StateStore stateStore;\n        private final MessageQueueListenerWrapper wrapper;\n        private final IdleWindowScaner idleWindowScaner;\n        private volatile boolean stop = false;\n\n        private Set<String> sourceTopicSet;\n\n        private final HashSet<MessageQueue> mq2Commit = new HashSet<>();\n\n\n        public PlanetaryEngine(DefaultLitePullConsumer unionConsumer, DefaultMQProducer producer, StateStore stateStore,\n                               DefaultMQAdminExt mqAdmin, MessageQueueListenerWrapper wrapper, Set<String> sourceTopicSet) {\n            this.unionConsumer = unionConsumer;\n            this.producer = producer;\n            this.mqAdmin = mqAdmin;\n            this.stateStore = stateStore;\n            this.wrapper = wrapper;\n            this.wrapper.setRecoverHandler((addQueue, removeQueue) -> {\n                try {\n                    PlanetaryEngine.this.stateStore.recover(addQueue, removeQueue);\n                    return null;\n                } catch (Throwable e) {\n                    logger.error(\"recover error.\", e);\n                    return e;\n                }\n            });\n            this.sourceTopicSet = sourceTopicSet;\n\n            Integer idleTime = (Integer) WorkerThread.this.properties.getOrDefault(StreamConfig.IDLE_TIME_TO_FIRE_WINDOW, 2000);\n            int commitInterval = (Integer) WorkerThread.this.properties.getOrDefault(StreamConfig.COMMIT_STATE_INTERNAL_MS, 2 * 1000);\n            this.idleWindowScaner = new IdleWindowScaner(idleTime, executor);\n            WorkerThread.this.executor.scheduleAtFixedRate(() -> {\n                try {\n                    doCommit(mq2Commit);\n                } catch (Throwable t) {\n                    logger.error(\"commit offset and state error.\", t);\n                }\n            }, 1000, commitInterval, TimeUnit.MILLISECONDS);\n        }\n\n\n        void start() throws Throwable {\n            createShuffleTopic();\n\n            this.unionConsumer.start();\n            this.producer.start();\n            this.stateStore.init();\n        }\n\n        void runInLoop() throws Throwable {\n            while (!stop) {\n                try {\n                    List<MessageExt> list = this.unionConsumer.poll(10);\n                    for (MessageExt messageExt : list) {\n                        byte[] body = messageExt.getBody();\n                        if (body == null || body.length == 0) {\n                            break;\n                        }\n\n                        String keyClassName = messageExt.getUserProperty(Constant.SHUFFLE_KEY_CLASS_NAME);\n                        String valueClassName = messageExt.getUserProperty(Constant.SHUFFLE_VALUE_CLASS_NAME);\n\n                        String topic = messageExt.getTopic();\n                        int queueId = messageExt.getQueueId();\n                        String brokerName = messageExt.getBrokerName();\n                        MessageQueue queue = new MessageQueue(topic, brokerName, queueId);\n                        mq2Commit.add(queue);\n                        logger.debug(\"source topic queue:[{}]\", queue);\n\n\n                        String key = Utils.buildKey(brokerName, topic, queueId);\n                        SourceSupplier.SourceProcessor<K, V> processor = (SourceSupplier.SourceProcessor<K, V>) wrapper.selectProcessor(key);\n\n                        StreamContextImpl<V> context = new StreamContextImpl<>(properties, producer, mqAdmin, stateStore, key, idleWindowScaner);\n\n                        processor.preProcess(context);\n\n                        Pair<K, V> pair = processor.deserialize(keyClassName, valueClassName, body);\n\n                        long timestamp = prepareTime(messageExt, processor);\n\n                        Data<K, V> data = new Data<>(pair.getKey(), pair.getValue(), timestamp, new Properties());\n                        context.setKey(pair.getKey());\n                        if (topic.contains(Constant.SHUFFLE_TOPIC_SUFFIX)) {\n                            logger.debug(\"shuffle data: [{}]\", data);\n                        } else {\n                            logger.debug(\"source data: [{}]\", data);\n                        }\n\n                        try {\n                            context.forward(data);\n                        } catch (Throwable t) {\n                            logger.error(\"process error.\", t);\n                            throw new DataProcessThrowable(t);\n                        }\n                    }\n\n                } catch (Throwable t) {\n                    Object skipDataError = properties.getOrDefault(Constant.SKIP_DATA_ERROR, Boolean.TRUE);\n                    if (skipDataError == Boolean.TRUE) {\n                        logger.error(\"ignore error, jobId=[{}], skip this data.\", topologyBuilder.getJobId(), t);\n                        //ignored\n                    } else {\n                        throw t;\n                    }\n                }\n            }\n        }\n\n        void doCommit(HashSet<MessageQueue> set) throws Throwable {\n            if (set != null && set.size() != 0) {\n\n                this.stateStore.persist(set);\n                this.unionConsumer.commit(set, true);\n\n                for (MessageQueue messageQueue : set) {\n                    logger.debug(\"committed messageQueue: [{}]\", messageQueue);\n                }\n\n                set.clear();\n            }\n        }\n\n        void maybeResetOffsetToFirst() throws Exception {\n            ConsumeFromWhere consumeFromWhere = (ConsumeFromWhere) properties.getOrDefault(ROCKETMQ_STREAMS_CONSUMER_FORM_WHERE, ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET);\n\n            if (!consumeFromWhere.equals(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET)) {\n                return;\n            }\n\n            for (String topic : sourceTopicSet) {\n                // 内部 topic 不能重置位点\n                if (topic.endsWith(Constant.SHUFFLE_TOPIC_SUFFIX) || topic.endsWith(STATE_TOPIC_SUFFIX)) {\n                    continue;\n                }\n                ConsumeStats consumeStats = mqAdmin.examineConsumeStats(unionConsumer.getConsumerGroup(), topic);\n                Map<MessageQueue, OffsetWrapper> offsetTable = consumeStats.getOffsetTable();\n                Set<MessageQueue> messageQueues = offsetTable.keySet();\n                for (MessageQueue messageQueue : messageQueues) {\n                    try {\n                        // 如果有消费进度，说明已经开始消费，跳过重置其消费进度\n                        if (offsetTable.containsKey(messageQueue) &&\n                                offsetTable.get(messageQueue).getConsumerOffset() != DEFAULT_CONSUME_OFFSET) {\n                            break;\n                        }\n\n                        Long minOffset = mqAdmin.minOffset(messageQueue);\n                        String brokerName = messageQueue.getBrokerName();\n                        ClusterInfo clusterInfo = mqAdmin.examineBrokerClusterInfo();\n                        BrokerData brokerData = clusterInfo.getBrokerAddrTable().get(brokerName);\n                        if (brokerData == null) {\n                            String msg = String.format(\"get broker error, have no broker info (name:%s)\", brokerName);\n                            logger.error(msg);\n                            throw new RStreamsException(msg);\n                        }\n                        for (String brokerAddress : brokerData.getBrokerAddrs().values()) {\n                            mqAdmin.resetOffsetByQueueId(brokerAddress,\n                                    unionConsumer.getConsumerGroup(),\n                                    messageQueue.getTopic(),\n                                    messageQueue.getQueueId(),\n                                    minOffset);\n                        }\n                    } catch (Exception e) {\n                        logger.error(\"reset messageQueue:{} consumer offset to first failed.\", messageQueue, e);\n                        throw e;\n                    }\n                }\n            }\n        }\n\n\n        long prepareTime(MessageExt messageExt, SourceSupplier.SourceProcessor<K, V> processor) {\n            TimeType type = (TimeType) properties.get(StreamConfig.TIME_TYPE);\n\n            long timestamp;\n            String userProperty = messageExt.getUserProperty(Constant.SOURCE_TIMESTAMP);\n            if (!StringUtils.isEmpty(userProperty)) {\n                //data come from shuffle topic\n                timestamp = Long.parseLong(userProperty);\n            } else {\n                //data come from user source topic\n                timestamp = processor.getTimestamp(messageExt, type);\n            }\n\n            return timestamp;\n        }\n\n        void createShuffleTopic() throws Throwable {\n            Set<String> total = WorkerThread.this.topologyBuilder.getSourceTopic();\n\n            List<String> shuffleTopic = new ArrayList<>();\n\n            for (String topic : total) {\n                if (topic.endsWith(Constant.SHUFFLE_TOPIC_SUFFIX)) {\n                    shuffleTopic.add(topic);\n                }\n            }\n\n            for (String topicName : shuffleTopic) {\n                RocketMQUtil.createStaticTopic(mqAdmin, topicName, StreamConfig.SHUFFLE_TOPIC_QUEUE_NUM);\n            }\n        }\n\n        public synchronized void stop() {\n            if (this.stop) {\n                return;\n            }\n\n            this.stop = true;\n\n            try {\n                this.unionConsumer.shutdown();\n\n                this.stateStore.close();\n                this.idleWindowScaner.close();\n\n                this.producer.shutdown();\n                this.mqAdmin.shutdown();\n                logger.info(\"shutdown engine success, thread:{}, jobId:{}\", WorkerThread.this.getName(), jobId);\n            } catch (Throwable e) {\n                logger.error(\"error when stop engin.\", e);\n                throw new RStreamsException(e);\n            }\n        }\n    }\n\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/serialization/KeyValueDeserializer.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.serialization;\n\n\nimport org.apache.rocketmq.streams.core.util.Pair;\n\npublic interface KeyValueDeserializer<K, V> {\n    default void configure(Object... args) throws Throwable {\n    }\n\n    Pair<K, V> deserialize(byte[] source) throws Throwable;\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/serialization/KeyValueSerializer.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.serialization;\n\npublic interface KeyValueSerializer<KEY, T> {\n    byte[] serialize(KEY key, T data) throws Throwable;\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/serialization/ShuffleProtocol.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.serialization;\n\nimport io.netty.buffer.ByteBuf;\nimport io.netty.buffer.ByteBufAllocator;\nimport io.netty.buffer.Unpooled;\nimport org.apache.rocketmq.streams.core.util.Pair;\n\n/**\n * shuffle data how to encode KV\n * <pre>\n * +-----------+---------------+-----------+-------------+\n * | Int(4)    | Int(4)        | key bytes | value bytes |\n * | key length| value length  |           |             |\n * +-----------+---------------+-----------+-------------+\n * </pre>\n */\npublic class ShuffleProtocol {\n    private final ByteBuf buf = Unpooled.buffer(16);\n    public byte[] merge(byte[] keyBytes, byte[] valueBytes) {\n        if (keyBytes == null || keyBytes.length ==0) {\n            return valueBytes;\n        }\n\n        if (valueBytes == null || valueBytes.length ==0) {\n            return keyBytes;\n        }\n\n        buf.writeInt(keyBytes.length);\n        buf.writeInt(valueBytes.length);\n        buf.writeBytes(keyBytes);\n        buf.writeBytes(valueBytes);\n\n        byte[] bytes = new byte[buf.readableBytes()];\n        buf.readBytes(bytes);\n\n        buf.clear();\n        return bytes;\n    }\n\n    public Pair<byte[], byte[]> split(byte[] total) {\n        ByteBuf byteBuf = Unpooled.wrappedBuffer(total);\n\n        int keyLength = byteBuf.readInt();\n        int valueLength = byteBuf.readInt();\n        ByteBuf keyByteBuf = byteBuf.readBytes(keyLength);\n        ByteBuf valueByteBuf = byteBuf.readBytes(valueLength);\n\n        byte[] keyBytes = new byte[keyByteBuf.readableBytes()];\n        keyByteBuf.readBytes(keyBytes);\n\n        byte[] valueBytes = new byte[valueByteBuf.readableBytes()];\n        valueByteBuf.readBytes(valueBytes);\n\n        byteBuf.release();\n        keyByteBuf.release();\n        valueByteBuf.release();\n        return new Pair<>(keyBytes, valueBytes);\n    }\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/serialization/deImpl/KVJsonDeserializer.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.serialization.deImpl;\n\nimport com.fasterxml.jackson.core.JsonGenerator;\nimport com.fasterxml.jackson.databind.DeserializationFeature;\nimport com.fasterxml.jackson.databind.ObjectMapper;\nimport com.fasterxml.jackson.databind.node.JsonNodeFactory;\nimport org.apache.commons.lang3.StringUtils;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueDeserializer;\nimport org.apache.rocketmq.streams.core.serialization.ShuffleProtocol;\n\npublic class KVJsonDeserializer<K, V> extends ShuffleProtocol implements KeyValueDeserializer<K, V> {\n    private final ObjectMapper objectMapper = new ObjectMapper();\n    private Class<K> keyType;\n    private Class<V> valueType;\n\n    public KVJsonDeserializer() {\n        objectMapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES)\n                .disable(DeserializationFeature.FAIL_ON_NULL_FOR_PRIMITIVES)\n                .enable(DeserializationFeature.ACCEPT_EMPTY_STRING_AS_NULL_OBJECT)\n                .enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS)\n                .enable(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN)\n                .setNodeFactory(JsonNodeFactory.withExactBigDecimals(true));\n\n    }\n\n    @Override\n    @SuppressWarnings(\"unchecked\")\n    public void configure(Object... args) throws Throwable {\n        String keyClassName = (String) args[0];\n        if (!StringUtils.isEmpty(keyClassName)) {\n            keyType = (Class<K>) Class.forName(keyClassName);\n        }\n\n        String valueClassName = (String) args[1];\n        if (!StringUtils.isEmpty(valueClassName)) {\n            valueType = (Class<V>) Class.forName(valueClassName);\n        }\n    }\n\n    @Override\n    public Pair<K, V> deserialize(byte[] total) throws Throwable {\n        Pair<byte[], byte[]> pair = split(total);\n\n        K key = null;\n        byte[] keyBytes = pair.getKey();\n        if (keyBytes != null && keyBytes.length != 0) {\n            key = objectMapper.readValue(keyBytes, keyType);\n        }\n\n        V value;\n        byte[] valueBytes = pair.getValue();\n        value = objectMapper.readValue(valueBytes, valueType);\n\n        return new Pair<>(key, value);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/serialization/serImpl/KVJsonSerializer.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.serialization.serImpl;\n\nimport com.fasterxml.jackson.core.JsonGenerator;\nimport com.fasterxml.jackson.databind.DeserializationFeature;\nimport com.fasterxml.jackson.databind.ObjectMapper;\nimport com.fasterxml.jackson.databind.node.JsonNodeFactory;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueSerializer;\nimport org.apache.rocketmq.streams.core.serialization.ShuffleProtocol;\n\npublic class KVJsonSerializer<K, V> extends ShuffleProtocol implements KeyValueSerializer<K, V> {\n    private final ObjectMapper objectMapper = new ObjectMapper();\n\n    public KVJsonSerializer() {\n        objectMapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES)\n                .disable(DeserializationFeature.FAIL_ON_NULL_FOR_PRIMITIVES)\n                .enable(DeserializationFeature.ACCEPT_EMPTY_STRING_AS_NULL_OBJECT)\n                .enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS)\n                .enable(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN)\n                .setNodeFactory(JsonNodeFactory.withExactBigDecimals(true));\n    }\n\n    @Override\n    public byte[] serialize(K key, V value) throws Throwable {\n        byte[] keyBytes;\n\n        if (key == null) {\n            keyBytes = new byte[0];\n        } else if (key instanceof byte[]) {\n            keyBytes = (byte[]) key;\n        } else {\n            keyBytes = objectMapper.writeValueAsBytes(key);\n        }\n\n        byte[] valueBytes;\n        if (value == null) {\n            valueBytes = new byte[0];\n        } else if (value instanceof byte[]) {\n            valueBytes = (byte[]) value;\n        } else {\n            valueBytes = objectMapper.writeValueAsBytes(value);\n        }\n\n\n        if (keyBytes.length == 0 && valueBytes.length == 0) {\n            return new byte[0];\n        }\n\n        return merge(keyBytes, valueBytes);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/state/AbstractStore.java",
    "content": "package org.apache.rocketmq.streams.core.state;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.commons.lang3.StringUtils;\nimport org.apache.rocketmq.common.message.MessageExt;\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.util.Utils;\n\nimport java.util.Arrays;\nimport java.util.HashSet;\nimport java.util.Iterator;\nimport java.util.Map;\nimport java.util.Set;\nimport java.util.concurrent.ConcurrentHashMap;\n\npublic abstract class AbstractStore {\n    private final Wrapper wrapper = new Wrapper();\n\n\n    protected void putInRecover(String stateTopicQueueKey, byte[] key) {\n        wrapper.putInRecover(stateTopicQueueKey, key);\n    }\n\n    protected void putInCalculating(String stateTopicQueueKey, byte[] key) {\n        wrapper.putInCalculating(stateTopicQueueKey, key);\n    }\n\n    protected Set<byte[]> getInCalculating(String stateTopicQueue) {\n        return wrapper.getInCalculating(stateTopicQueue);\n    }\n\n    protected void removeCalculating(String stateTopicQueue) {\n        wrapper.removeCalculating(stateTopicQueue);\n    }\n\n    protected Set<byte[]> getAll(String stateTopicQueue) {\n        return wrapper.getAll(stateTopicQueue);\n    }\n\n\n    protected String whichStateTopicQueueBelongTo(byte[] key) {\n        return wrapper.whichStateTopicQueueBelongTo(key);\n    }\n\n    protected void removeAllKey(byte[] key) {\n        wrapper.deleteByKey(key);\n    }\n\n\n    protected void removeAll(String stateTopicQueue) {\n        wrapper.removeAll(stateTopicQueue);\n    }\n\n    protected MessageQueue convertSourceTopicQueue2StateTopicQueue(MessageQueue messageQueue) {\n        HashSet<MessageQueue> messageQueues = new HashSet<>();\n        messageQueues.add(messageQueue);\n\n        Set<MessageQueue> stateTopicQueue = convertSourceTopicQueue2StateTopicQueue(messageQueues);\n\n        Iterator<MessageQueue> iterator = stateTopicQueue.iterator();\n        return iterator.next();\n    }\n\n    protected Set<MessageQueue> convertSourceTopicQueue2StateTopicQueue(Set<MessageQueue> messageQueues) {\n        if (messageQueues == null || messageQueues.size() == 0) {\n            return new HashSet<>();\n        }\n\n        HashSet<MessageQueue> result = new HashSet<>();\n        for (MessageQueue messageQueue : messageQueues) {\n            if (messageQueue.getTopic().endsWith(Constant.STATE_TOPIC_SUFFIX)) {\n                result.add(messageQueue);\n                continue;\n            }\n            MessageQueue queue = new MessageQueue(messageQueue.getTopic() + Constant.STATE_TOPIC_SUFFIX, messageQueue.getBrokerName(), messageQueue.getQueueId());\n            result.add(queue);\n        }\n\n        return result;\n    }\n\n    protected static String stateTopic2SourceTopic(String stateTopic) {\n        if (StringUtils.isEmpty(stateTopic)) {\n            return null;\n        }\n\n        return stateTopic.substring(0, stateTopic.lastIndexOf(Constant.STATE_TOPIC_SUFFIX));\n    }\n\n\n    protected String buildKey(MessageExt messageExt) {\n        return Utils.buildKey(messageExt.getBrokerName(), messageExt.getTopic(), messageExt.getQueueId());\n    }\n\n    protected String buildKey(MessageQueue messageQueue) {\n        return Utils.buildKey(messageQueue.getBrokerName(), messageQueue.getTopic(), messageQueue.getQueueId());\n    }\n\n    static class Wrapper {\n        //新增，写消费未提交保存的中间状态，提交时移除\n        private final ConcurrentHashMap<String/*brokerName@topic@queueId of state topic*/, Set<byte[]/*Key*/>> calculating = new ConcurrentHashMap<>();\n        //全量, 与rocksdb保持同步\n        private final ConcurrentHashMap<String/*brokerName@topic@queueId of state topic*/, Set<byte[]/*Key*/>> recover = new ConcurrentHashMap<>();\n\n        public void putInRecover(String stateTopicQueueKey, byte[] key) {\n            Set<byte[]> allSet = this.recover.computeIfAbsent(stateTopicQueueKey, s -> new HashSet<>());\n\n            for (byte[] item : allSet) {\n                if (Arrays.equals(item, key)) {\n                    return;\n                }\n            }\n\n            allSet.add(key);\n        }\n\n        public void putInCalculating(String stateTopicQueueKey, byte[] key) {\n            Set<byte[]> keySet = this.calculating.computeIfAbsent(stateTopicQueueKey, s -> new HashSet<>());\n\n            for (byte[] item : keySet) {\n                if (Arrays.equals(item, key)) {\n                    return;\n                }\n            }\n\n            keySet.add(key);\n\n            putInRecover(stateTopicQueueKey, key);\n        }\n\n        public Set<byte[]> getInCalculating(String stateTopicQueue) {\n            return calculating.getOrDefault(stateTopicQueue, new HashSet<>());\n        }\n\n        public Set<byte[]> getAll(String stateTopicQueue) {\n            Set<byte[]> calculating = this.calculating.getOrDefault(stateTopicQueue, new HashSet<>());\n            Set<byte[]> recover = this.recover.getOrDefault(stateTopicQueue, new HashSet<>());\n\n            Set<byte[]> result = new HashSet<>();\n            result.addAll(calculating);\n            result.addAll(recover);\n\n            //可能有重复，不同byte[] 但是时一个key\n            return result;\n        }\n\n        public String whichStateTopicQueueBelongTo(byte[] key) {\n            for (String uniqueQueue : recover.keySet()) {\n                for (byte[] tempKeyByte : recover.getOrDefault(uniqueQueue, new HashSet<>())) {\n                    if (Arrays.equals(tempKeyByte, key)) {\n                        return uniqueQueue;\n                    }\n                }\n            }\n\n            for (String uniqueQueue : calculating.keySet()) {\n                for (byte[] tempKeyByte : calculating.getOrDefault(uniqueQueue, new HashSet<>())) {\n                    if (Arrays.equals(tempKeyByte, key)) {\n                        return uniqueQueue;\n                    }\n                }\n            }\n\n            return null;\n        }\n\n\n        public void deleteByKey(byte[] key) {\n            {\n                Set<Map.Entry<String, Set<byte[]>>> entries = calculating.entrySet();\n                Iterator<Map.Entry<String, Set<byte[]>>> iterator = entries.iterator();\n                while (iterator.hasNext()) {\n                    Map.Entry<String, Set<byte[]>> next = iterator.next();\n\n                    Set<byte[]> keySet = next.getValue();\n\n                    if (keySet != null) {\n                        keySet.removeIf(rocksDBKey -> Arrays.equals(rocksDBKey, key));\n                        if (keySet.size() == 0) {\n                            iterator.remove();\n                        }\n                    }\n                }\n            }\n\n            {\n                Set<Map.Entry<String, Set<byte[]>>> entries = recover.entrySet();\n                Iterator<Map.Entry<String, Set<byte[]>>> iterator = entries.iterator();\n                while (iterator.hasNext()) {\n                    Map.Entry<String, Set<byte[]>> next = iterator.next();\n\n                    Set<byte[]> keySet = next.getValue();\n\n                    if (keySet != null) {\n                        keySet.removeIf(rocksDBKey -> Arrays.equals(rocksDBKey, key));\n                        if (keySet.size() == 0) {\n                            iterator.remove();\n                        }\n                    }\n                }\n            }\n\n\n\n        }\n\n        public void removeCalculating(String stateTopicQueueKey) {\n            this.calculating.remove(stateTopicQueueKey);\n        }\n\n        public void removeAll(String stateTopicQueueKey) {\n            this.recover.remove(stateTopicQueueKey);\n            this.calculating.remove(stateTopicQueueKey);\n        }\n\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/state/RocketMQStore.java",
    "content": "package org.apache.rocketmq.streams.core.state;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.commons.lang3.StringUtils;\nimport org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;\nimport org.apache.rocketmq.client.producer.DefaultMQProducer;\nimport org.apache.rocketmq.common.CountDownLatch2;\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.common.consumer.ConsumeFromWhere;\nimport org.apache.rocketmq.common.message.Message;\nimport org.apache.rocketmq.common.message.MessageExt;\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.common.protocol.route.BrokerData;\nimport org.apache.rocketmq.common.protocol.route.QueueData;\nimport org.apache.rocketmq.common.protocol.route.TopicRouteData;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.metadata.StreamConfig;\nimport org.apache.rocketmq.streams.core.util.ColumnFamilyUtil;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.serialization.ShuffleProtocol;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.RocketMQUtil;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.tools.admin.DefaultMQAdminExt;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.nio.charset.StandardCharsets;\nimport java.util.ArrayList;\nimport java.util.Comparator;\nimport java.util.List;\nimport java.util.Map;\nimport java.util.Properties;\nimport java.util.Set;\nimport java.util.concurrent.ConcurrentHashMap;\nimport java.util.concurrent.ExecutorService;\nimport java.util.concurrent.Executors;\nimport java.util.concurrent.Future;\nimport java.util.concurrent.TimeUnit;\nimport java.util.concurrent.TimeoutException;\nimport java.util.stream.Collectors;\n\npublic class RocketMQStore extends AbstractStore implements StateStore {\n    private static final Logger logger = LoggerFactory.getLogger(RocketMQStore.class.getName());\n    private final DefaultMQProducer producer;\n    private final DefaultMQAdminExt mqAdmin;\n    private final RocksDBStore rocksDBStore;\n    private final Properties properties;\n\n    private final ExecutorService executor = Executors.newFixedThreadPool(8);\n    private final ShuffleProtocol protocol = new ShuffleProtocol();\n\n    private final ConcurrentHashMap<MessageQueue/*messageQueue of state topic*/, CountDownLatch2> recoveringQueueMutex = new ConcurrentHashMap<>();\n\n    public RocketMQStore(DefaultMQProducer producer, RocksDBStore rocksDBStore, DefaultMQAdminExt mqAdmin, Properties properties) {\n        this.producer = producer;\n        this.mqAdmin = mqAdmin;\n        this.rocksDBStore = rocksDBStore;\n        this.properties = properties;\n    }\n\n    @Override\n    public void init() throws Throwable {\n    }\n\n    @Override\n    public void recover(Set<MessageQueue> addQueues, Set<MessageQueue> removeQueues) throws Throwable {\n        this.loadState(addQueues);\n        this.removeState(removeQueues);\n    }\n\n    @Override\n    public void waitIfNotReady(MessageQueue messageQueue) throws RecoverStateStoreThrowable {\n        MessageQueue stateTopicQueue = convertSourceTopicQueue2StateTopicQueue(messageQueue);\n        CountDownLatch2 waitPoint = this.recoveringQueueMutex.get(stateTopicQueue);\n\n        long start = 0;\n        long end = 0;\n        try {\n            start = System.currentTimeMillis();\n            waitPoint.await(5000, TimeUnit.MILLISECONDS);\n            end = System.currentTimeMillis();\n        } catch (Throwable t) {\n            throw new RecoverStateStoreThrowable(t);\n        } finally {\n            long cost = end - start;\n            if (cost > 2000) {\n                logger.error(\"recover finish, consume time:\" + cost + \" ms.\");\n            }\n        }\n    }\n\n\n    @Override\n    public byte[] get(String columnFamily, byte[] key) throws Throwable {\n        if (key == null || key.length == 0) {\n            return new byte[0];\n        }\n        return this.rocksDBStore.get(columnFamily, key);\n    }\n\n    @Override\n    public void put(MessageQueue stateTopicMessageQueue, String columnFamily, byte[] key, byte[] value) throws Throwable {\n        String stateTopicQueueKey = buildKey(stateTopicMessageQueue);\n        super.putInCalculating(stateTopicQueueKey, key);\n        this.rocksDBStore.put(columnFamily, key, value);\n    }\n\n    @Override\n    public List<Pair<byte[], byte[]>> searchStateLessThanWatermark(String keyPrefix, long lessThanThisTime, ValueMapperAction<byte[], WindowKey> deserializer) throws Throwable {\n        if (StringUtils.isEmpty(keyPrefix)) {\n            return new ArrayList<>();\n        }\n\n        return this.rocksDBStore.searchStateLessThanWatermark(keyPrefix, lessThanThisTime, deserializer);\n    }\n\n    @Override\n    public List<Pair<String, byte[]>> searchByKeyPrefix(String keyPrefix,\n                                                        ValueMapperAction<String, byte[]> string2Bytes,\n                                                        ValueMapperAction<byte[], String> byte2String) throws Throwable {\n        if (StringUtils.isEmpty(keyPrefix)) {\n            return new ArrayList<>();\n        }\n        return this.rocksDBStore.searchByKeyPrefix(keyPrefix, string2Bytes, byte2String);\n    }\n\n    @Override\n    public void delete(byte[] key) throws Throwable {\n        if (key == null || key.length == 0) {\n            return;\n        }\n        //删除远程\n        String stateTopicQueue = super.whichStateTopicQueueBelongTo(key);\n        String[] split = Utils.split(stateTopicQueue);\n        String topic = split[1];\n        MessageQueue queue = new MessageQueue(split[1], split[0], Integer.parseInt(split[2]));\n\n        Message message = new Message(topic, Constant.EMPTY_BODY.getBytes(StandardCharsets.UTF_8));\n        message.setKeys(Utils.toHexString(key));\n        message.putUserProperty(Constant.SHUFFLE_KEY_CLASS_NAME, key.getClass().getName());\n        message.putUserProperty(Constant.EMPTY_BODY, Constant.TRUE);\n        producer.send(message, queue);\n\n        //删除rocksdb\n        this.rocksDBStore.deleteByKey(ColumnFamilyUtil.getColumnFamilyByKey(key), key);\n\n        //删除内存中的key\n        super.removeAllKey(key);\n\n        logger.debug(\"delete key from RocketMQ and Rocksdb, key=\" + new String(key, StandardCharsets.UTF_8) + \",MessageQueue: \" + queue);\n    }\n\n    @Override\n    public void persist(Set<MessageQueue> messageQueues) throws Throwable {\n        if (messageQueues == null || messageQueues.size() == 0) {\n            return;\n        }\n\n        Set<MessageQueue> stateTopicQueues = convertSourceTopicQueue2StateTopicQueue(messageQueues);\n        for (MessageQueue stateTopicQueue : stateTopicQueues) {\n            String stateTopicQueueKey = buildKey(stateTopicQueue);\n            Set<byte[]> keySet = super.getInCalculating(stateTopicQueueKey);\n\n            if (keySet == null || keySet.size() == 0) {\n                continue;\n            }\n\n            String stateTopic = stateTopicQueue.getTopic();\n            boolean isStaticTopic = stateTopicQueue.getBrokerName().equals(Constant.STATIC_TOPIC_BROKER_NAME);\n            createStateTopic(stateTopic, isStaticTopic);\n\n            for (byte[] key : keySet) {\n\n                byte[] valueBytes = this.rocksDBStore.get(ColumnFamilyUtil.getColumnFamilyByKey(key), key);\n                if (valueBytes == null) {\n                    continue;\n                }\n\n                byte[] body = this.protocol.merge(key, valueBytes);\n\n                Message message = new Message(stateTopicQueue.getTopic(), body);\n                message.setKeys(Utils.toHexString(key));\n\n                try {\n                    logger.debug(\"persist key: \" + new String(key, StandardCharsets.UTF_8) + \",messageQueue: \" + stateTopicQueue);\n                } catch (Throwable t) {\n                    //key is not string, maybe.\n                }\n\n                this.producer.send(message, stateTopicQueue);\n            }\n            super.removeCalculating(stateTopicQueueKey);\n        }\n    }\n\n    public void loadState(Set<MessageQueue> addQueues) throws Throwable {\n        if (addQueues == null || addQueues.size() == 0) {\n            return;\n        }\n\n        final DefaultLitePullConsumer consumer = new DefaultLitePullConsumer(StreamConfig.ROCKETMQ_STREAMS_STATE_CONSUMER_GROUP);\n        consumer.setNamesrvAddr(properties.getProperty(MixAll.NAMESRV_ADDR_PROPERTY));\n        consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET);\n        consumer.setAutoCommit(false);\n        consumer.start();\n\n        Set<MessageQueue> stateTopicQueue = convertSourceTopicQueue2StateTopicQueue(addQueues);\n        for (MessageQueue messageQueue : stateTopicQueue) {\n            createStateTopic(messageQueue.getTopic(), messageQueue.getBrokerName().equals(Constant.STATIC_TOPIC_BROKER_NAME));\n        }\n\n        consumer.assign(stateTopicQueue);\n        for (MessageQueue queue : stateTopicQueue) {\n            consumer.seekToBegin(queue);\n        }\n\n        Future<?> future = this.executor.submit(() -> {\n            try {\n                pullToLast(consumer);\n            } catch (Throwable e) {\n                logger.error(\"pull to last error.\", e);\n                throw new RuntimeException(e);\n            } finally {\n                consumer.shutdown();\n            }\n        });\n\n        try {\n            future.get(100, TimeUnit.MILLISECONDS);\n        } catch (InterruptedException | TimeoutException e) {\n        }\n    }\n\n    public void removeState(Set<MessageQueue> removeQueues) throws Throwable {\n        if (removeQueues == null || removeQueues.size() == 0) {\n            return;\n        }\n\n        Future<?> future = this.executor.submit(() -> {\n            try {\n                if (removeQueues.size() == 0) {\n                    return;\n                }\n                Set<MessageQueue> stateTopicQueue = convertSourceTopicQueue2StateTopicQueue(removeQueues);\n\n                Map<String/*brokerName@topic@queueId*/, List<MessageQueue>> groupByUniqueQueue = stateTopicQueue.stream().parallel().collect(Collectors.groupingBy(this::buildKey));\n                for (String stateUniqueQueue : groupByUniqueQueue.keySet()) {\n                    Set<byte[]> stateTopicQueueKey = super.getAll(stateUniqueQueue);\n                    for (byte[] key : stateTopicQueueKey) {\n                        this.rocksDBStore.deleteByKey(ColumnFamilyUtil.getColumnFamilyByKey(key), key);\n                    }\n                    super.removeAll(stateUniqueQueue);\n                }\n\n\n                for (MessageQueue stateMessageQueue : stateTopicQueue) {\n                    this.recoveringQueueMutex.remove(stateMessageQueue);\n                }\n            } catch (Throwable e) {\n                logger.error(\"remove state error\", e);\n                throw new RuntimeException(e);\n            }\n        });\n\n        try {\n            future.get(100, TimeUnit.MILLISECONDS);\n        } catch (InterruptedException | TimeoutException e) {\n        }\n    }\n\n    private void pullToLast(DefaultLitePullConsumer consumer) throws Throwable {\n        Set<MessageQueue> readyToRecover = consumer.assignment();\n        for (MessageQueue messageQueue : readyToRecover) {\n            this.recoveringQueueMutex.computeIfAbsent(messageQueue, messageQueue1 -> new CountDownLatch2(1));\n        }\n\n        List<MessageExt> holder = new ArrayList<>();\n        //recover\n        List<MessageExt> result = consumer.poll(50);\n        while (result != null && result.size() != 0) {\n            holder.addAll(result);\n            if (holder.size() <= 1000) {\n                result = consumer.poll(50);\n                continue;\n            }\n\n            replayState(holder);\n            holder.clear();\n\n            result = consumer.poll(50);\n        }\n        if (holder.size() != 0) {\n            replayState(holder);\n        }\n\n        //恢复完毕；\n        Set<MessageQueue> recoverOver = consumer.assignment();\n        for (MessageQueue messageQueue : recoverOver) {\n            CountDownLatch2 waitPoint = this.recoveringQueueMutex.get(messageQueue);\n            waitPoint.countDown();\n        }\n    }\n\n    //拉的数据越多，重放效率越高,\n    // 能保证一个q里面后面pull到的数据queueOffset一定比前一批次拉取的queueOffset大吗？\n    private void replayState(List<MessageExt> msgs) throws Throwable {\n        if (msgs == null || msgs.size() == 0) {\n            return;\n        }\n\n        Map<String/*brokerName@topic@queueId of state topic*/, List<MessageExt>> groupByQueueId = msgs.stream().parallel().collect(Collectors.groupingBy(this::buildKey));\n\n        for (String uniqueQueue : groupByQueueId.keySet()) {\n            List<MessageExt> messageExts = groupByQueueId.get(uniqueQueue);\n            Map<String/*K的hashcode，真正的key在body里面*/, List<MessageExt>> groupByKeyHashcode = messageExts.stream().parallel().collect(Collectors.groupingBy(MessageExt::getKeys));\n\n            for (String keyHashcode : groupByKeyHashcode.keySet()) {\n                //相同brokerName@topic@queueId + keyHashcode 在一次拉取中的所有数据\n                List<MessageExt> exts = groupByKeyHashcode.get(keyHashcode);\n\n                //取最大queueOffset的消息，按照queueOffset，相同key，大的queueOffset覆盖小的queueOffset\n                MessageExt result = exts.stream()\n                        .max(Comparator.comparingLong(MessageExt::getQueueOffset))\n                        .orElse(null);\n\n                if (result == null) {\n                    continue;\n                }\n\n                String emptyBody = result.getUserProperty(Constant.EMPTY_BODY);\n                if (Constant.TRUE.equals(emptyBody)) {\n                    continue;\n                }\n\n                byte[] body = result.getBody();\n                Pair<byte[], byte[]> pair = this.protocol.split(body);\n\n                byte[] key = pair.getKey();\n                byte[] value = pair.getValue();\n\n                //放入rocksdb\n                MessageQueue stateTopicQueue = new MessageQueue(result.getTopic(), result.getBrokerName(), result.getQueueId());\n                try {\n                    logger.debug(\"recover state, key: \" + new String(key, StandardCharsets.UTF_8) + \", stateTopicQueue: \" + stateTopicQueue);\n                } catch (Throwable t) {\n                }\n\n                String stateTopicQueueKey = buildKey(stateTopicQueue);\n                super.putInRecover(stateTopicQueueKey, key);\n                this.rocksDBStore.put(ColumnFamilyUtil.getColumnFamilyByKey(key), key, value);\n            }\n        }\n    }\n\n\n    private void createStateTopic(String stateTopic, boolean sourceTopicIsStaticTopic) throws Exception {\n        if (RocketMQUtil.checkWhetherExist(stateTopic)) {\n            return;\n        }\n\n        String sourceTopic = stateTopic2SourceTopic(stateTopic);\n        Pair<Integer, Set<String>> clustersPair = getTotalQueueNumAndClusters(sourceTopic);\n\n        if (sourceTopicIsStaticTopic) {\n            RocketMQUtil.createStaticCompactTopic(mqAdmin, stateTopic, clustersPair.getKey(), clustersPair.getValue());\n        } else {\n            RocketMQUtil.createNormalTopic(mqAdmin, sourceTopic, stateTopic);\n        }\n    }\n\n    private Pair<Integer, Set<String>> getTotalQueueNumAndClusters(String sourceTopic) throws Exception {\n        int queueNum = 0;\n\n        //找到brokerAddr\n        TopicRouteData topicRouteData = mqAdmin.examineTopicRouteInfo(sourceTopic);\n        List<QueueData> queueData = topicRouteData.getQueueDatas();\n\n        List<BrokerData> brokerData = topicRouteData.getBrokerDatas();\n        Set<String> clusterSet = brokerData.stream().collect(Collectors.groupingBy(BrokerData::getCluster)).keySet();\n\n        for (QueueData data : queueData) {\n            //只看readQueue\n            queueNum += data.getReadQueueNums();\n        }\n\n        return new Pair<Integer, Set<String>>(queueNum, clusterSet);\n    }\n\n    @Override\n    public void close() throws Exception {\n        this.rocksDBStore.close();\n        this.executor.shutdown();\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/state/RocksDBStore.java",
    "content": "package org.apache.rocketmq.streams.core.state;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n\nimport org.apache.commons.io.FileUtils;\nimport org.apache.commons.lang3.StringUtils;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.util.ColumnFamilyUtil;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.rocksdb.*;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.io.File;\nimport java.io.IOException;\nimport java.nio.charset.StandardCharsets;\nimport java.util.ArrayList;\nimport java.util.List;\n\npublic class RocksDBStore extends AbstractStore implements AutoCloseable {\n    private static final Logger logger = LoggerFactory.getLogger(RocksDBStore.class);\n\n    private static final String ROCKSDB_PATH = \"/tmp/rocksdb\";\n    private RocksDB rocksDB;\n    private WriteOptions writeOptions;\n    private ReadOptions readOptions;\n    private File storeFile;\n\n    public RocksDBStore(String path) {\n        createRocksDB(path);\n    }\n\n    private void createRocksDB(String path) {\n        try (final Options options = new Options().setCreateIfMissing(true)) {\n\n            try {\n                String rocksdbFilePath = String.format(\"%s/%s\", ROCKSDB_PATH, path);\n\n                storeFile = new File(rocksdbFilePath);\n\n                if (storeFile.exists()) {\n                    FileUtils.forceDelete(storeFile);\n                }\n\n                if (!storeFile.mkdirs()) {\n                    throw new RuntimeException(\"before create rocksdb,mkdir path \" + rocksdbFilePath + \" error\");\n                }\n\n                this.rocksDB = TtlDB.open(options, rocksdbFilePath, 10800, false);\n                ColumnFamilyUtil.createColumnFamilies(this.rocksDB, new ColumnFamilyOptions());\n\n                writeOptions = new WriteOptions();\n                writeOptions.setSync(false);\n                writeOptions.setDisableWAL(true);\n            } catch (RocksDBException e) {\n                throw new RuntimeException(\"create rocksdb error \" + e.getMessage());\n            } catch (IOException e) {\n                throw new RuntimeException(\"delete rocksdb directory:\" + ROCKSDB_PATH + \"field.\");\n            }\n        }\n    }\n\n    public byte[] get(String columnFamilyName, byte[] key) throws RocksDBException {\n        if (key == null) {\n            return null;\n        }\n\n        return rocksDB.get(ColumnFamilyUtil.getColumnFamilyHandleByName(columnFamilyName), key);\n    }\n\n    public void put(String columnFamilyName, byte[] key, byte[] value) throws RocksDBException {\n        rocksDB.put(ColumnFamilyUtil.getColumnFamilyHandleByName(columnFamilyName), writeOptions, key, value);\n    }\n\n    public List<Pair<byte[], byte[]>> searchStateLessThanWatermark(String name,\n                                                                   long lessThanThisTime,\n                                                                   ValueMapperAction<byte[], WindowKey> deserializer) throws Throwable {\n        readOptions = new ReadOptions();\n        readOptions.setPrefixSameAsStart(true).setTotalOrderSeek(true);\n\n        RocksIterator rocksIterator = rocksDB.newIterator(ColumnFamilyUtil.getColumnFamilyHandleByName(ColumnFamilyUtil.WINDOW_STATE_CF), readOptions);\n        byte[] keyBytePrefix = name.getBytes(StandardCharsets.UTF_8);\n        rocksIterator.seek(keyBytePrefix);\n\n        List<Pair<byte[], byte[]>> temp = new ArrayList<>();\n        while (rocksIterator.isValid()) {\n            byte[] keyBytes = rocksIterator.key();\n            byte[] valueBytes = rocksIterator.value();\n            rocksIterator.next();\n\n            WindowKey windowKey = deserializer.convert(keyBytes);\n            if (!windowKey.getOperatorName().equals(name)) {\n                continue;\n            }\n\n            if (windowKey.getWindowEnd() >= lessThanThisTime) {\n                continue;\n            }\n\n            Pair<byte[], byte[]> pair = new Pair<>(keyBytes, valueBytes);\n            temp.add(pair);\n        }\n        return temp;\n    }\n\n    public List<Pair<String, byte[]>> searchByKeyPrefix(String keyPrefix,\n                                                        ValueMapperAction<String, byte[]> string2Bytes,\n                                                        ValueMapperAction<byte[], String> byte2String) throws Throwable {\n        readOptions = new ReadOptions();\n        readOptions.setPrefixSameAsStart(true).setTotalOrderSeek(true);\n        RocksIterator rocksIterator = rocksDB.newIterator(readOptions);\n\n        byte[] convert = string2Bytes.convert(keyPrefix);\n        rocksIterator.seek(convert);\n\n        List<Pair<String, byte[]>> temp = new ArrayList<>();\n        while (rocksIterator.isValid()) {\n            byte[] keyBytes = rocksIterator.key();\n            byte[] valueBytes = rocksIterator.value();\n\n            if (skipWatermarkKey(keyBytes)) {\n                continue;\n            }\n\n            String storeKey = byte2String.convert(keyBytes);\n            if (storeKey.startsWith(keyPrefix)) {\n                Pair<String, byte[]> pair = new Pair<>(storeKey, valueBytes);\n                temp.add(pair);\n            }\n\n            rocksIterator.next();\n        }\n\n        return temp;\n    }\n\n    public void deleteByKey(String columnFamilyName, byte[] key) throws RocksDBException {\n        rocksDB.delete(ColumnFamilyUtil.getColumnFamilyHandleByName(columnFamilyName), key);\n    }\n\n    public void close() throws Exception {\n        this.rocksDB.close();\n        if (this.storeFile != null && storeFile.exists()) {\n            FileUtils.forceDelete(storeFile);\n            logger.info(\"close RocksDB success, delete path:{}\", storeFile.getPath());\n        }\n    }\n\n    //todo: column family to solve this problem.\n    private boolean skipWatermarkKey(byte[] target) {\n        if (target == null || target.length == 0) {\n            return false;\n        }\n\n        try {\n            String key = new String(target, StandardCharsets.UTF_8);\n\n            return !StringUtils.isBlank(key) && key.startsWith(Constant.WATERMARK_KEY);\n        } catch (Throwable ignored) {\n            return false;\n        }\n    }\n\n\n    public static void main(String[] args) throws Throwable {\n        RocksDBStore rocksDBStore = new RocksDBStore(\"test\");\n\n        String key = \"time@1668249210000@1668249195000\";\n        String key2 = \"time@1668249210001@1668249195001\";\n        Object value = \"3\";\n        Object value2 = \"2\";\n\n        byte[] keyBytes = Utils.object2Byte(key);\n        byte[] valueBytes = Utils.object2Byte(value);\n\n        byte[] keyBytes2 = Utils.object2Byte(key2);\n        byte[] valueBytes2 = Utils.object2Byte(value2);\n\n        rocksDBStore.put(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes2), keyBytes2, valueBytes2);\n        rocksDBStore.put(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes), keyBytes, valueBytes);\n\n\n        byte[] bytes = rocksDBStore.get(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes), keyBytes);\n        Object result = Utils.byte2Object(bytes, Object.class);\n        System.out.println(result);\n\n        byte[] bytes2 = rocksDBStore.get(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes2), keyBytes2);\n        Object result2 = Utils.byte2Object(bytes2, Object.class);\n        System.out.println(result2);\n\n        String keyPrefix = \"time@1668249210000\";\n\n\n        List<Pair<String, byte[]>> pairs = rocksDBStore.searchByKeyPrefix(keyPrefix, Utils::object2Byte, data -> Utils.byte2Object(data, String.class));\n        for (Pair<String, byte[]> pair : pairs) {\n            assert pair.getKey().startsWith(keyPrefix);\n        }\n\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/state/StateStore.java",
    "content": "package org.apache.rocketmq.streams.core.state;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.util.Pair;\n\nimport java.util.List;\nimport java.util.Set;\n\npublic interface StateStore extends AutoCloseable {\n    void init() throws Throwable;\n\n\n    //addQueues    messageQueue of source topic,removeQueues messageQueue of source topic\n    void recover(Set<MessageQueue> addQueues, Set<MessageQueue> removeQueues) throws Throwable;\n\n\n    //messageQueue check the state of source topic is ok, wait if not.\n    void waitIfNotReady(MessageQueue messageQueue) throws RecoverStateStoreThrowable;\n\n\n    byte[] get(String columnFamily, byte[] key) throws Throwable;\n\n    void put(MessageQueue stateTopicMessageQueue, String columnFamily, byte[] key, byte[] value) throws Throwable;\n\n    List<Pair<byte[], byte[]>> searchStateLessThanWatermark(String operatorName, long lessThanThisTime, ValueMapperAction<byte[], WindowKey> deserializer) throws Throwable;\n\n    List<Pair<String, byte[]>> searchByKeyPrefix(String keyPrefix, ValueMapperAction<String, byte[]> string2Bytes, ValueMapperAction<byte[], String> byte2String) throws Throwable;\n\n    void delete(byte[] key) throws Throwable;\n\n    void persist(Set<MessageQueue> messageQueue) throws Throwable;\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/TopologyBuilder.java",
    "content": "package org.apache.rocketmq.streams.core.topology;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.topology.real.ProcessorFactory;\nimport org.apache.rocketmq.streams.core.topology.real.RealProcessorFactory;\nimport org.apache.rocketmq.streams.core.topology.real.SinkFactory;\nimport org.apache.rocketmq.streams.core.topology.real.SourceFactory;\n\nimport java.util.ArrayList;\nimport java.util.Collections;\nimport java.util.HashMap;\nimport java.util.LinkedHashMap;\nimport java.util.List;\nimport java.util.Set;\nimport java.util.function.Supplier;\n\npublic class TopologyBuilder {\n    private final String jobId;\n\n    private final LinkedHashMap<String/*source topic*/, SourceFactory<?>> topic2SourceNodeFactory = new LinkedHashMap<>();\n\n    private final LinkedHashMap<String/*name*/, RealProcessorFactory<?>> realNodeFactory = new LinkedHashMap<>();\n\n    private final LinkedHashMap<String/*sink topic*/, RealProcessorFactory<?>> topic2SinkNodeFactory = new LinkedHashMap<>();\n\n    private final HashMap<String/*source name*/, List<String/*subsequent processor without source*/>> source2Group = new HashMap<>();\n\n    public TopologyBuilder(String jobId) {\n        this.jobId = jobId;\n    }\n\n    public <T> void addRealSource(String name, String topicName, Supplier<Processor<T>> supplier) {\n        SourceFactory<T> sourceFactory = new SourceFactory<>(name, topicName, supplier);\n\n        realNodeFactory.put(name, sourceFactory);\n\n        topic2SourceNodeFactory.put(topicName, sourceFactory);\n\n        //将source与sink之间的节点分为一个组，处理数据时，不同分组使用不同task\n        source2Group.put(name, new ArrayList<>());\n    }\n\n\n    public <T> void addRealNode(String name, String parentName, Supplier<? extends Processor<T>> supplier) {\n        RealProcessorFactory<T> processorFactory = new ProcessorFactory<>(name, supplier);\n        realNodeFactory.put(name, processorFactory);\n\n        grouping(name, parentName);\n    }\n\n\n\n\n    public <T> void addRealSink(String name, String parentName, String topicName, Supplier<Processor<T>> supplier) {\n        SinkFactory<T> sinkFactory = new SinkFactory<>(name, supplier);\n        realNodeFactory.put(name, sinkFactory);\n        topic2SinkNodeFactory.put(topicName, sinkFactory);\n        grouping(name, parentName);\n    }\n\n\n    private void grouping(String name, String parentName) {\n        if (source2Group.containsKey(parentName)) {\n            source2Group.get(parentName).add(name);\n        } else {\n            for (String sourceName : source2Group.keySet()) {\n                List<String> subsequentProcessor = source2Group.get(sourceName);\n                if (subsequentProcessor.contains(parentName)) {\n                    subsequentProcessor.add(name);\n                }\n            }\n        }\n    }\n\n\n    public Set<String> getSourceTopic() {\n        return Collections.unmodifiableSet(this.topic2SourceNodeFactory.keySet());\n    }\n\n\n\n    @SuppressWarnings(\"unchecked\")\n    public <T> Processor<T> build(String topicName) {\n        SourceFactory<T> sourceFactory = (SourceFactory<T>) topic2SourceNodeFactory.get(topicName);\n        Processor<T> sourceProcessor = sourceFactory.build();\n\n        String sourceName = sourceFactory.getName();\n\n        //集合中的顺序就是算子的父子顺序，前面的是后面的父亲节点\n        List<String> groupNames = source2Group.get(sourceName);\n\n        Processor<T> parent = sourceProcessor;\n        for (String child : groupNames) {\n            RealProcessorFactory<T> childProcessorFactory = (RealProcessorFactory<T>) realNodeFactory.get(child);\n            Processor<T> childProcessor = childProcessorFactory.build();\n            parent.addChild(childProcessor);\n            parent = childProcessor;\n        }\n\n        return sourceProcessor;\n    }\n\n    public String getJobId() {\n        return jobId;\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/real/ProcessorFactory.java",
    "content": "package org.apache.rocketmq.streams.core.topology.real;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n\nimport org.apache.rocketmq.streams.core.running.Processor;\n\nimport java.util.ArrayList;\nimport java.util.List;\nimport java.util.function.Supplier;\n\npublic class ProcessorFactory<T> implements RealProcessorFactory<T> {\n    private final String name;\n    private final Supplier<? extends Processor<T>> supplier;\n\n    public ProcessorFactory(String name, Supplier<? extends Processor<T>> supplier) {\n        this.name = name;\n        this.supplier = supplier;\n    }\n\n    @Override\n    public String getName() {\n        return name;\n    }\n\n    @Override\n    public Processor<T> build() {\n        return supplier.get();\n    }\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/real/RealProcessorFactory.java",
    "content": "package org.apache.rocketmq.streams.core.topology.real;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.running.Processor;\n\nimport java.util.List;\n\npublic interface RealProcessorFactory<T> {\n    String getName();\n\n    Processor<T> build();\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/real/SinkFactory.java",
    "content": "package org.apache.rocketmq.streams.core.topology.real;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.running.Processor;\n\nimport java.util.ArrayList;\nimport java.util.List;\nimport java.util.function.Supplier;\n\npublic class SinkFactory<T> implements RealProcessorFactory<T> {\n    private final String name;\n    private final List<RealProcessorFactory<T>> children = new ArrayList<>();\n    private final Supplier<? extends Processor<T>> supplier;\n\n    public SinkFactory(String name, Supplier<? extends Processor<T>> supplier) {\n        this.name = name;\n        this.supplier = supplier;\n    }\n\n\n    @Override\n    public String getName() {\n        return name;\n    }\n\n    @Override\n    public Processor<T> build() {\n        return supplier.get();\n    }\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/real/SourceFactory.java",
    "content": "package org.apache.rocketmq.streams.core.topology.real;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.running.Processor;\n\nimport java.util.ArrayList;\nimport java.util.Collections;\nimport java.util.List;\nimport java.util.function.Supplier;\n\npublic class SourceFactory<T> implements RealProcessorFactory<T> {\n    private final String name;\n    private final String topic;\n    private final Supplier<Processor<T>> supplier;\n\n    public SourceFactory(String name, String topic, Supplier<Processor<T>> supplier) {\n        this.name = name;\n        this.topic = topic;\n        this.supplier = supplier;\n    }\n\n    @Override\n    public String getName() {\n        return name;\n    }\n\n    @Override\n    public Processor<T> build() {\n        return supplier.get();\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/AbstractGraphNode.java",
    "content": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.Collections;\nimport java.util.List;\nimport java.util.Objects;\n\npublic abstract class AbstractGraphNode implements GraphNode {\n    private static final Logger logger = LoggerFactory.getLogger(AbstractGraphNode.class.getName());\n\n    private final List<GraphNode> parents = new ArrayList<>();\n    private final List<GraphNode> children = new ArrayList<>();\n\n    protected String name;\n\n\n    public AbstractGraphNode(String name) {\n        Objects.requireNonNull(name, \"name can not be null.\");\n        this.name = name;\n    }\n\n\n    @Override\n    public String getName() {\n        return name;\n    }\n\n    @Override\n    public void addParent(GraphNode graphNode) {\n        if (!parents.contains(graphNode)) {\n            parents.add(graphNode);\n        } else {\n            logger.error(\"GraphNode: [\" + graphNode + \"] has exist in parent set.\");\n        }\n    }\n\n    @Override\n    public void addChild(GraphNode graphNode) {\n        if (!children.contains(graphNode)) {\n            children.add(graphNode);\n        } else {\n            logger.error(\"GraphNode: [\" + graphNode + \"] has exist in children set.\");\n        }\n    }\n\n\n    @Override\n    public List<GraphNode> getAllChild() {\n        return Collections.unmodifiableList(this.children);\n    }\n\n    @Override\n    public List<GraphNode> getAllParent() {\n        return Collections.unmodifiableList(this.parents);\n    }\n\n    @Override\n    public boolean shuffleNode() {\n        return false;\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/GraphNode.java",
    "content": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\n\nimport java.util.List;\n\npublic interface GraphNode {\n    String getName();\n\n    void addParent(GraphNode graphNode);\n\n\n    void addChild(GraphNode graphNode);\n\n    List<GraphNode> getAllChild();\n\n    List<GraphNode> getAllParent();\n\n    boolean shuffleNode();\n\n\n    void addRealNode(TopologyBuilder builder);\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/ProcessorNode.java",
    "content": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\n\nimport java.util.ArrayList;\nimport java.util.List;\nimport java.util.function.Supplier;\n\npublic class ProcessorNode<T> extends AbstractGraphNode {\n    protected final Supplier<Processor<T>> supplier;\n    protected final List<String> parentNames;\n    protected boolean shuffle = false;\n\n\n    public ProcessorNode(String name, String parentName, Supplier<Processor<T>> supplier) {\n        super(name);\n        this.supplier = supplier;\n        this.parentNames = new ArrayList<>();\n        this.parentNames.add(parentName);\n    }\n\n    public ProcessorNode(String name, List<String> parentNames, Supplier<Processor<T>> supplier) {\n        super(name);\n        this.supplier = supplier;\n        this.parentNames = parentNames;\n    }\n\n    public ProcessorNode(String name, List<String> parentNames, boolean shuffle, Supplier<Processor<T>> supplier) {\n        super(name);\n        this.supplier = supplier;\n        this.parentNames = parentNames;\n        this.shuffle = shuffle;\n    }\n\n    public ProcessorNode(String name, String parentName, boolean shuffle, Supplier<Processor<T>> supplier) {\n        super(name);\n        this.supplier = supplier;\n        this.parentNames = new ArrayList<>();\n        this.parentNames.add(parentName);\n        this.shuffle = shuffle;\n    }\n\n    @Override\n    public boolean shuffleNode() {\n        return this.shuffle;\n    }\n\n\n    @Override\n    public void addRealNode(TopologyBuilder builder) {\n        //parentName数量大于2时，可能因为有的parent所在source并没有准备好，source2Group没有对应parent，\n        // 无法添加导致该流最后缺少window以后的算子\n        for (String parentName : parentNames) {\n            builder.addRealNode(name, parentName, supplier);\n        }\n    }\n\n    @Override\n    public boolean equals(Object o) {\n        if (this == o) return true;\n        if (o == null || getClass() != o.getClass()) return false;\n        ProcessorNode<?> that = (ProcessorNode<?>) o;\n        return this.name.equals(that.name);\n    }\n\n    @Override\n    public String toString() {\n        return \"ProcessorNode{\" + \"name=[\" + name + \"]}\";\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/ShuffleProcessorNode.java",
    "content": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.util.OperatorNameMaker;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.function.supplier.SinkSupplier;\nimport org.apache.rocketmq.streams.core.function.supplier.SourceSupplier;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.serialization.deImpl.KVJsonDeserializer;\nimport org.apache.rocketmq.streams.core.serialization.serImpl.KVJsonSerializer;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport java.util.function.Supplier;\n\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SHUFFLE_SINK_PREFIX;\nimport static org.apache.rocketmq.streams.core.util.OperatorNameMaker.SHUFFLE_SOURCE_PREFIX;\n\npublic class ShuffleProcessorNode<T> extends ProcessorNode<T> {\n\n\n    public ShuffleProcessorNode(String name, String parentName, Supplier<Processor<T>> supplier) {\n        super(name, parentName, supplier);\n    }\n\n    @Override\n    public void addRealNode(TopologyBuilder builder) {\n        String topicName = name + Constant.SHUFFLE_TOPIC_SUFFIX;\n        //join的时候需要一个相同的topicName\n\n        String shuffleSinkName = OperatorNameMaker.makeName(SHUFFLE_SINK_PREFIX, builder.getJobId());\n        for (String parentName : parentNames) {\n            builder.addRealSink(shuffleSinkName, parentName, topicName, new SinkSupplier<>(topicName, new KVJsonSerializer<>()));\n        }\n        String shuffleSourceName = OperatorNameMaker.makeName(SHUFFLE_SOURCE_PREFIX, builder.getJobId());\n        builder.addRealSource(shuffleSourceName, topicName, new SourceSupplier<>(topicName, new KVJsonDeserializer<>()));\n\n        builder.addRealNode(name, shuffleSourceName, supplier);\n    }\n\n    @Override\n    public String toString() {\n        return \"ShuffleProcessorNode{\" + \"name=[\" + name + \"]}\";\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/SinkGraphNode.java",
    "content": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\n\nimport java.util.function.Supplier;\n\npublic class SinkGraphNode<T> extends AbstractGraphNode {\n    private final Supplier<Processor<T>> supplier;\n    private final String topicName;\n    private final String parentName;\n\n    public SinkGraphNode(String name, String parentName, String topicName, Supplier<Processor<T>> supplier) {\n        super(name);\n        this.topicName = topicName;\n        this.supplier = supplier;\n        this.parentName = parentName;\n    }\n\n\n    @Override\n    public void addRealNode(TopologyBuilder builder) {\n        builder.addRealSink(name, parentName, topicName, supplier);\n    }\n\n    @Override\n    public String toString() {\n        return \"SinkGraphNode{\" + \"name=[\" + name + \"]}\";\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/topology/virtual/SourceGraphNode.java",
    "content": "package org.apache.rocketmq.streams.core.topology.virtual;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport org.apache.rocketmq.streams.core.function.supplier.SourceSupplier;\nimport org.apache.rocketmq.streams.core.running.Processor;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueDeserializer;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\n\nimport java.util.function.Supplier;\n\npublic class SourceGraphNode<T> extends AbstractGraphNode {\n    private Supplier<Processor<T>> supplier;\n    private String topicName;\n\n\n    public SourceGraphNode(String name, String topicName, KeyValueDeserializer<Void, T> deserializer) {\n        super(name);\n        this.topicName = topicName;\n        this.supplier = new SourceSupplier<>(topicName, deserializer);\n    }\n\n    @Override\n    public void addRealNode(TopologyBuilder builder) {\n        builder.addRealSource(name, topicName, supplier);\n    }\n\n    @Override\n    public String toString() {\n        return \"SourceGraphNode{\" + \"name=[\" + name + \"]}\";\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/util/ColumnFamilyUtil.java",
    "content": "package org.apache.rocketmq.streams.core.util;\n\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.rocksdb.*;\n\nimport java.io.UnsupportedEncodingException;\nimport java.nio.charset.StandardCharsets;\nimport java.util.*;\nimport java.util.function.Predicate;\n\npublic class ColumnFamilyUtil {\n\n    public static final String WATERMARK_STATE_CF = \"watermark-state\";\n\n    public static final String WINDOW_STATE_CF = \"window-state\";\n\n    public static final String VALUE_STATE_CF = \"value-state\";\n\n    private enum ColumnFamilyEnum {\n        WATERMARK_STATE_COLUMN_FAMILY(WATERMARK_STATE_CF, str -> {\n            return str.startsWith(Constant.WATERMARK_KEY);\n        }),\n        WINDOW_STATE_COLUMN_FAMILY(WINDOW_STATE_CF, str -> {\n            String[] splits = str.split(\"&&\");\n            return splits.length == 4;\n        }),\n        VALUE_STATE_COLUMN_FAMILY(VALUE_STATE_CF, str -> true)\n        ;\n\n        private final String name;\n\n        private final Predicate<String> keyCheckFunc;\n\n        ColumnFamilyEnum(String name, Predicate<String> keyCheckFunc) {\n            this.name = name;\n            this.keyCheckFunc = keyCheckFunc;\n        }\n\n    }\n\n    private final static Map<String, ColumnFamilyHandle> cFName2CFHandle = new HashMap<>();\n\n    public static void createColumnFamilies(RocksDB rocksDB, ColumnFamilyOptions cfOpts) throws RocksDBException, UnsupportedEncodingException {\n        final List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();\n        for (ColumnFamilyEnum columnFamilyEnum : ColumnFamilyEnum.values()) {\n            cfDescriptors.add(new ColumnFamilyDescriptor(columnFamilyEnum.name.getBytes(StandardCharsets.UTF_8), cfOpts));\n        }\n        List<ColumnFamilyHandle> columnFamilyHandles = rocksDB.createColumnFamilies(cfDescriptors);\n        for (ColumnFamilyHandle columnFamilyHandle : columnFamilyHandles) {\n            cFName2CFHandle.put(new String(columnFamilyHandle.getName(), StandardCharsets.UTF_8), columnFamilyHandle);\n        }\n    }\n\n    public static String getColumnFamilyByKey(byte[] key) {\n        if (key == null) {\n            return null;\n        }\n        return getColumnFamilyByKey(new String(key, StandardCharsets.UTF_8));\n    }\n\n    public static String getColumnFamilyByKey(String key) {\n        for (ColumnFamilyEnum columnFamilyEnum : ColumnFamilyEnum.values()) {\n            if (columnFamilyEnum.keyCheckFunc.test(key)) {\n                return columnFamilyEnum.name;\n            }\n        }\n        return null;\n    }\n\n    public static ColumnFamilyHandle getColumnFamilyHandleByName(String name) {\n        return cFName2CFHandle.get(name);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/util/OperatorNameMaker.java",
    "content": "package org.apache.rocketmq.streams.core.util;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport java.util.concurrent.atomic.AtomicInteger;\n\npublic class OperatorNameMaker {\n    public static final String SOURCE_PREFIX = \"ROCKETMQ-SOURCE\";\n    public static final String SHUFFLE_SOURCE_PREFIX = \"ROCKETMQ-SHUFFLE-SOURCE\";\n    public static final String SINK_PREFIX = \"ROCKETMQ-SINK\";\n    public static final String PRINT_PREFIX = \"ROCKETMQ-PRINT\";\n    public static final String SHUFFLE_SINK_PREFIX = \"ROCKETMQ-SHUFFLE-SINK\";\n    public static final String MAP_PREFIX = \"ROCKETMQ-MAP\";\n    public static final String FLAT_MAP_PREFIX = \"ROCKETMQ-FLATMAP\";\n    public static final String FILTER_PREFIX = \"ROCKETMQ-FILTER\";\n    public static final String GROUPBY_PREFIX = \"ROCKETMQ-GROUPBY\";\n    public static final String COUNT_PREFIX = \"ROCKETMQ-COUNT\";\n    public static final String MIN_PREFIX = \"ROCKETMQ-MIN\";\n    public static final String MAX_PREFIX = \"ROCKETMQ-MAX\";\n    public static final String SUM_PREFIX = \"ROCKETMQ-SUM\";\n    public static final String FOR_EACH_PREFIX = \"ROCKETMQ-FOREACH\";\n    public static final String WINDOW_PREFIX = \"ROCKETMQ-WINDOW\";\n    public static final String ADD_TAG = \"ROCKETMQ-ADD-TAG\";\n    public static final String WINDOW_AVG_PREFIX = \"ROCKETMQ-WINDOW-AVG\";\n    public static final String AGGREGATE_PREFIX = \"ROCKETMQ-AGGREGATE\";\n    public static final String ACCUMULATE_PREFIX = \"ROCKETMQ-ACCUMULATE\";\n    public static final String JOIN_WINDOW_PREFIX = \"JOIN-WINDOW\";\n    public static final String JOIN_PREFIX = \"JOIN\";\n\n    public static final String pattern = \"%s-%s-%s\";\n\n    private static final ThreadLocal<AtomicInteger> index = ThreadLocal.withInitial(() -> new AtomicInteger(0));\n\n\n    private static int incrementAndGet() {\n        return index.get().incrementAndGet();\n    }\n\n    public static String makeName(String prefix, String jobId) {\n        String number = String.format(\"%05d\", incrementAndGet());\n\n        return String.format(pattern, jobId, prefix, number);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/util/Pair.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.util;\n\n\nimport com.fasterxml.jackson.annotation.JsonCreator;\nimport com.fasterxml.jackson.annotation.JsonProperty;\n\npublic class Pair<T1, T2> {\n    private T1 key;\n    private T2 value;\n\n    @JsonCreator\n    public Pair(@JsonProperty(\"key\") T1 key, @JsonProperty(\"value\") T2 value) {\n        this.key = key;\n        this.value = value;\n    }\n\n    public T1 getKey() {\n        return key;\n    }\n\n    public void setKey(T1 key) {\n        this.key = key;\n    }\n\n    public T2 getValue() {\n        return value;\n    }\n\n    public void setValue(T2 value) {\n        this.value = value;\n    }\n\n    @Override\n    public String toString() {\n        return \"Pair{\" +\n                \"object1=\" + key +\n                \", object2=\" + value +\n                '}';\n    }\n}"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/util/RocketMQUtil.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.util;\n\nimport org.apache.commons.cli.CommandLine;\nimport org.apache.commons.cli.Options;\nimport org.apache.commons.cli.PosixParser;\nimport org.apache.rocketmq.client.exception.MQClientException;\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.common.TopicConfig;\nimport org.apache.rocketmq.common.constant.PermName;\nimport org.apache.rocketmq.common.protocol.ResponseCode;\nimport org.apache.rocketmq.common.protocol.body.ClusterInfo;\nimport org.apache.rocketmq.common.protocol.route.BrokerData;\nimport org.apache.rocketmq.common.protocol.route.QueueData;\nimport org.apache.rocketmq.common.protocol.route.TopicRouteData;\nimport org.apache.rocketmq.remoting.exception.RemotingException;\nimport org.apache.rocketmq.srvutil.ServerUtil;\nimport org.apache.rocketmq.tools.admin.DefaultMQAdminExt;\nimport org.apache.rocketmq.tools.command.CommandUtil;\nimport org.apache.rocketmq.tools.command.topic.UpdateStaticTopicSubCommand;\nimport org.apache.rocketmq.tools.command.topic.UpdateTopicSubCommand;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.HashMap;\nimport java.util.HashSet;\nimport java.util.List;\nimport java.util.Set;\n\npublic class RocketMQUtil {\n    private static final Logger logger = LoggerFactory.getLogger(RocketMQUtil.class.getName());\n\n    private static final List<String> existTopic = new ArrayList<>();\n\n    //neither static topic nor compact topic. expansion with source topic.\n    public static void createNormalTopic(DefaultMQAdminExt mqAdmin, String topicName, int totalQueueNum, Set<String> clusters) throws Exception {\n        if (check(mqAdmin, topicName)) {\n            logger.info(\"topic[{}] already exist.\", topicName);\n            return;\n        }\n\n        if (clusters == null || clusters.size() == 0) {\n            clusters = getCluster(mqAdmin);\n        }\n\n\n        for (String cluster : clusters) {\n            Set<String> masterSet = CommandUtil.fetchMasterAddrByClusterName(mqAdmin, cluster);\n\n            int remainder = totalQueueNum % masterSet.size();\n            if (remainder != 0) {\n                String temp = String.format(\"can not create topic:%s, total num=%s, master num=%s\", topicName, totalQueueNum, masterSet.size());\n                logger.warn(temp);\n            }\n\n            int queueNumInEachBroker = totalQueueNum / masterSet.size();\n            TopicConfig topicConfig = new TopicConfig(topicName, queueNumInEachBroker, queueNumInEachBroker, PermName.PERM_READ | PermName.PERM_WRITE);\n\n            if (remainder == 0) {\n                for (String addr : masterSet) {\n                    mqAdmin.createAndUpdateTopicConfig(addr, topicConfig);\n                    logger.info(\"create topic to broker:{} cluster:{}, success.\", addr, cluster);\n                }\n            } else {\n                String[] masterArray = masterSet.toArray(new String[]{});\n\n                topicConfig = new TopicConfig(topicName, queueNumInEachBroker + remainder,\n                        queueNumInEachBroker + remainder, PermName.PERM_READ | PermName.PERM_WRITE);\n                mqAdmin.createAndUpdateTopicConfig(masterArray[0], topicConfig);\n\n                for (int i = 1; i < masterArray.length; i++) {\n                    topicConfig = new TopicConfig(topicName, queueNumInEachBroker, queueNumInEachBroker, PermName.PERM_READ | PermName.PERM_WRITE);\n                    mqAdmin.createAndUpdateTopicConfig(masterArray[0], topicConfig);\n                }\n            }\n\n        }\n    }\n\n    public static void createNormalTopic(DefaultMQAdminExt mqAdmin, String sourceTopic, String stateTopic) throws Exception {\n        //找到brokerAddr\n        TopicRouteData topicRouteData = mqAdmin.examineTopicRouteInfo(sourceTopic);\n        List<QueueData> queueData = topicRouteData.getQueueDatas();\n        List<BrokerData> brokerData = topicRouteData.getBrokerDatas();\n\n\n        HashMap<String, String> brokerName2MaterBrokerAddr = new HashMap<>();\n        for (BrokerData broker : brokerData) {\n            String masterBrokerAddr = broker.getBrokerAddrs().get(0L);\n            brokerName2MaterBrokerAddr.put(broker.getBrokerName(), masterBrokerAddr);\n        }\n\n        for (QueueData queue : queueData) {\n            int readQueueNums = queue.getReadQueueNums();\n            int writeQueueNums = queue.getWriteQueueNums();\n            String brokerName = queue.getBrokerName();\n\n            TopicConfig topicConfig = new TopicConfig(stateTopic, readQueueNums, writeQueueNums);\n\n            mqAdmin.createAndUpdateTopicConfig(brokerName2MaterBrokerAddr.get(brokerName), topicConfig);\n        }\n\n        existTopic.add(stateTopic);\n    }\n\n    //used in RSQLDB,maybe.\n    public static void createStaticCompactTopic(DefaultMQAdminExt mqAdmin, String topicName, int totalQueueNum, Set<String> clusters) throws Exception {\n        if (check(mqAdmin, topicName)) {\n            logger.info(\"topic[{}] already exist.\", topicName);\n            return;\n        }\n\n        if (clusters == null || clusters.size() == 0) {\n            clusters = getCluster(mqAdmin);\n        }\n\n\n        for (String cluster : clusters) {\n            createStaticTopicWithCommand(topicName, totalQueueNum, new HashSet<>(), cluster, mqAdmin.getNamesrvAddr());\n            logger.info(\"【step 1】create static topic:[{}] in cluster:[{}] success, logic queue num:[{}].\", topicName, cluster, totalQueueNum);\n\n            update2CompactTopicWithCommand(topicName, totalQueueNum, cluster, mqAdmin.getNamesrvAddr());\n            logger.info(\"【step 2】update static topic to compact topic success. topic:[{}], cluster:[{}]\", topicName, cluster);\n        }\n\n        existTopic.add(topicName);\n        logger.info(\"create static-compact topic [{}] success, queue num [{}]\", topicName, totalQueueNum);\n    }\n\n    public static void createStaticTopic(DefaultMQAdminExt mqAdmin, String topicName, int queueNum) throws Exception {\n        if (check(mqAdmin, topicName)) {\n            logger.info(\"topic[{}] already exist.\", topicName);\n            return;\n        }\n\n        Set<String> clusters = getCluster(mqAdmin);\n        for (String cluster : clusters) {\n            createStaticTopicWithCommand(topicName, queueNum, new HashSet<>(), cluster, mqAdmin.getNamesrvAddr());\n            logger.info(\"create static topic:[{}] in cluster:[{}] success, logic queue num:[{}].\", topicName, cluster, queueNum);\n        }\n\n        existTopic.add(topicName);\n    }\n\n\n    private static void createStaticTopicWithCommand(String topic, int totalQueueNum, Set<String> brokers, String cluster, String nameservers) throws Exception {\n        UpdateStaticTopicSubCommand cmd = new UpdateStaticTopicSubCommand();\n        Options options = ServerUtil.buildCommandlineOptions(new Options());\n        String[] args;\n        if (cluster != null) {\n            args = new String[]{\n                    \"-c\", cluster,\n                    \"-t\", topic,\n                    \"-qn\", String.valueOf(totalQueueNum),\n                    \"-n\", nameservers\n            };\n        } else {\n            String brokerStr = String.join(\",\", brokers);\n            args = new String[]{\n                    \"-b\", brokerStr,\n                    \"-t\", topic,\n                    \"-qn\", String.valueOf(totalQueueNum),\n                    \"-n\", nameservers\n            };\n        }\n\n        final CommandLine commandLine = ServerUtil.parseCmdLine(\"mqadmin \" + cmd.commandName(), args, cmd.buildCommandlineOptions(options), new PosixParser());\n\n        String namesrvAddr = commandLine.getOptionValue('n');\n        System.setProperty(MixAll.NAMESRV_ADDR_PROPERTY, namesrvAddr);\n\n        cmd.execute(commandLine, options, null);\n    }\n\n    private static void update2CompactTopicWithCommand(String topic, int queueNum, String cluster, String nameservers) throws Exception {\n        UpdateTopicSubCommand command = new UpdateTopicSubCommand();\n        Options options = ServerUtil.buildCommandlineOptions(new Options());\n        String[] args = new String[]{\n                \"-c\", cluster,\n                \"-t\", topic,\n                \"-r\", String.valueOf(queueNum),\n                \"-w\", String.valueOf(queueNum),\n                \"-n\", nameservers\n//                todo 发布版本还不支持\n//                , \"-a\", \"+delete.policy=COMPACTION\"\n        };\n\n        final CommandLine commandLine = ServerUtil.parseCmdLine(\"mqadmin \" + command.commandName(), args, command.buildCommandlineOptions(options), new PosixParser());\n        String namesrvAddr = commandLine.getOptionValue('n');\n        System.setProperty(MixAll.NAMESRV_ADDR_PROPERTY, namesrvAddr);\n\n        command.execute(commandLine, options, null);\n    }\n\n\n    public static Set<String> getCluster(DefaultMQAdminExt mqAdmin) throws Exception {\n        ClusterInfo clusterInfo = mqAdmin.examineBrokerClusterInfo();\n        return clusterInfo.getClusterAddrTable().keySet();\n    }\n\n    private static boolean check(DefaultMQAdminExt mqAdmin, String topicName) {\n        if (existTopic.contains(topicName)) {\n            return true;\n        }\n\n        try {\n            mqAdmin.examineTopicRouteInfo(topicName);\n            existTopic.add(topicName);\n            return true;\n        } catch (RemotingException | InterruptedException e) {\n            logger.error(\"examine topic route info error.\", e);\n            throw new RuntimeException(\"examine topic route info error.\", e);\n        } catch (MQClientException exception) {\n            if (exception.getResponseCode() == ResponseCode.TOPIC_NOT_EXIST) {\n                logger.info(\"topic[{}] does not exist, create it.\", topicName);\n            } else {\n                throw new RuntimeException(exception);\n            }\n        }\n        return false;\n    }\n\n    public static boolean checkWhetherExist(String topic) {\n        return existTopic.contains(topic);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/util/Utils.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.util;\n\n\nimport com.fasterxml.jackson.core.JsonGenerator;\nimport com.fasterxml.jackson.core.JsonProcessingException;\nimport com.fasterxml.jackson.databind.DeserializationFeature;\nimport com.fasterxml.jackson.databind.ObjectMapper;\nimport com.fasterxml.jackson.databind.node.JsonNodeFactory;\nimport org.apache.commons.codec.digest.DigestUtils;\nimport org.apache.commons.lang3.StringUtils;\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RStreamsException;\n\nimport java.io.IOException;\nimport java.io.InputStream;\nimport java.nio.ByteBuffer;\nimport java.nio.charset.StandardCharsets;\nimport java.text.SimpleDateFormat;\nimport java.util.Date;\n\npublic class Utils {\n    private static final ObjectMapper objectMapper = new ObjectMapper();\n    public static final String pattern = \"%s@%s@%s\";\n\n    static {\n        objectMapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES)\n                .disable(DeserializationFeature.FAIL_ON_NULL_FOR_PRIMITIVES)\n                .enable(DeserializationFeature.ACCEPT_EMPTY_STRING_AS_NULL_OBJECT)\n                .enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS)\n                .enable(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN)\n                .setNodeFactory(JsonNodeFactory.withExactBigDecimals(true));\n    }\n\n    public static String buildKey(String brokerName, String topic, int queueId) {\n        return String.format(pattern, brokerName, topic, queueId);\n    }\n\n    public static String buildKey(String key, String... args) {\n        if (StringUtils.isEmpty(key)) {\n            return null;\n        }\n\n        StringBuilder builder = new StringBuilder();\n        builder.append(key);\n\n        if (args == null || args.length == 0) {\n            return builder.toString();\n        }\n\n        builder.append(Constant.SPLIT);\n        for (String arg : args) {\n            builder.append(arg);\n            builder.append(Constant.SPLIT);\n        }\n\n        return builder.substring(0, builder.lastIndexOf(Constant.SPLIT));\n    }\n\n    public static String[] split(String source) {\n        return split(source, Constant.SPLIT);\n    }\n\n    public static String[] split(String source, String split) {\n        if (StringUtils.isEmpty(source) || StringUtils.isEmpty(split)) {\n            return new String[]{};\n        }\n\n        return source.split(split);\n    }\n\n    public static byte[] object2Byte(Object target) throws JsonProcessingException {\n        if (target == null) {\n            return new byte[]{};\n        }\n\n        return objectMapper.writeValueAsBytes(target);\n    }\n\n\n    public static <B> B byte2Object(byte[] source, Class<B> clazz) throws IOException {\n        if (source == null || source.length == 0 || clazz == null) {\n            return null;\n        }\n\n        return objectMapper.readValue(source, clazz);\n    }\n\n    public static SimpleDateFormat df = new SimpleDateFormat(\"yyyy-MM-dd HH:mm:ss\");\n\n    public static String format(long timestamp) {\n        Date date = new Date(timestamp);\n        return df.format(date);\n    }\n\n    public static String toHexString(Object obj) {\n        try {\n            if (obj instanceof byte[]) {\n                return DigestUtils.md5Hex((byte[]) obj);\n            } else if (obj instanceof String) {\n                return DigestUtils.md5Hex((String) obj);\n            } else if (obj instanceof InputStream) {\n                return DigestUtils.md5Hex((InputStream) obj);\n            } else {\n                byte[] bytes = object2Byte(obj);\n                return DigestUtils.md5Hex(bytes);\n            }\n        } catch (Throwable t) {\n            throw new RStreamsException(\"object to HexString error, object=\" + obj, t);\n        }\n\n    }\n\n\n    public static byte[] long2Bytes(long time) {\n        ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES);\n        buffer.putLong(time);\n        return buffer.array();\n    }\n\n    public static long bytes2Long(byte[] bytes) {\n        if (bytes == null || bytes.length == 0) {\n            return 0;\n        }\n\n        ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES);\n        buffer.put(bytes);\n        buffer.flip();//need flip\n        return buffer.getLong();\n    }\n\n    public static byte[] watermarkKeyBytes(MessageQueue stateTopicMessageQueue, String watermarkPrefix) {\n        if (stateTopicMessageQueue == null || StringUtils.isBlank(watermarkPrefix)) {\n            throw new IllegalArgumentException();\n        }\n\n        String key = Utils.buildKey(watermarkPrefix,\n                stateTopicMessageQueue.getBrokerName(),\n                stateTopicMessageQueue.getTopic(),\n                String.valueOf(stateTopicMessageQueue.getQueueId()));\n\n        assert key != null;\n        return key.getBytes(StandardCharsets.UTF_8);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/JoinType.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window;\n\npublic enum JoinType {\n    INNER_JOIN,\n    LEFT_JOIN\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/StreamType.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window;\n\npublic enum StreamType {\n    LEFT_STREAM,\n    RIGHT_STREAM\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/Time.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window;\n\nimport java.util.concurrent.TimeUnit;\n\nimport static io.netty.util.internal.ObjectUtil.checkNotNull;\n\npublic class Time {\n    private final TimeUnit unit;\n    private final long size;\n\n\n    private Time(long size, TimeUnit unit) {\n        this.unit = checkNotNull(unit, \"time unit may not be null\");\n        this.size = size;\n    }\n\n    public TimeUnit getUnit() {\n        return unit;\n    }\n\n\n    public long getSize() {\n        return size;\n    }\n\n\n    public long toMilliseconds() {\n        return unit.toMillis(size);\n    }\n\n    public long toMillSecond() {\n        return this.unit.toMillis(this.size);\n    }\n\n    public static Time of(long size, TimeUnit unit) {\n        return new Time(size, unit);\n    }\n\n    public static Time milliseconds(long milliseconds) {\n        return of(milliseconds, TimeUnit.MILLISECONDS);\n    }\n\n    public static Time seconds(long seconds) {\n        return of(seconds, TimeUnit.SECONDS);\n    }\n\n    public static Time minutes(long minutes) {\n        return of(minutes, TimeUnit.MINUTES);\n    }\n\n    public static Time hours(long hours) {\n        return of(hours, TimeUnit.HOURS);\n    }\n\n    public static Time days(long days) {\n        return of(days, TimeUnit.DAYS);\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/TimeType.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window;\n\npublic enum TimeType {\n    EVENT_TIME,\n    PROCESS_TIME\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/Window.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window;\n\nimport org.apache.rocketmq.streams.core.util.Utils;\n\npublic class Window {\n    private long startTime;\n\n    private long endTime;\n\n    public Window(long startTime, long endTime) {\n        this.startTime = startTime;\n        this.endTime = endTime;\n    }\n\n    public long getStartTime() {\n        return startTime;\n    }\n\n    public void setStartTime(long startTime) {\n        this.startTime = startTime;\n    }\n\n    public long getEndTime() {\n        return endTime;\n    }\n\n    public void setEndTime(long endTime) {\n        this.endTime = endTime;\n    }\n\n\n    @Override\n    public String toString() {\n\n        return \"[\" +\n                \"startTime=\" + Utils.format(startTime) +\n                \",\" +\n                \"endTime=\" + Utils.format(endTime) +\n                \"]\";\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/WindowBuilder.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window;\n\n\npublic class WindowBuilder {\n\n    public static WindowInfo tumblingWindow(Time size) {\n        WindowInfo info = new WindowInfo();\n\n        info.setWindowType(WindowInfo.WindowType.TUMBLING_WINDOW);\n        info.setWindowSize(size);\n        info.setWindowSlide(size);\n\n        return info;\n    }\n\n    public static WindowInfo slidingWindow(Time size, Time slideStep) {\n        WindowInfo info = new WindowInfo();\n\n        info.setWindowType(WindowInfo.WindowType.SLIDING_WINDOW);\n        info.setWindowSize(size);\n        info.setWindowSlide(slideStep);\n\n        return info;\n    }\n\n    public static WindowInfo sessionWindow(Time size) {\n        WindowInfo info = new WindowInfo();\n\n        info.setWindowType(WindowInfo.WindowType.SESSION_WINDOW);\n        info.setSessionTimeout(size);\n\n        return info;\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/WindowInfo.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window;\n\n\npublic class WindowInfo {\n    private WindowType windowType;\n\n    private JoinStream joinStream = null;\n\n    private Time windowSize;//窗口大小\n\n    private Time windowSlide;//滑动大小\n\n    private Time sessionTimeout;\n\n\n    public WindowType getWindowType() {\n        return windowType;\n    }\n\n    public void setWindowType(WindowType windowType) {\n        this.windowType = windowType;\n    }\n\n    public Time getWindowSize() {\n        return windowSize;\n    }\n\n    public void setWindowSize(Time windowSize) {\n        this.windowSize = windowSize;\n    }\n\n    public Time getWindowSlide() {\n        return windowSlide;\n    }\n\n    public void setWindowSlide(Time windowSlide) {\n        this.windowSlide = windowSlide;\n    }\n\n    public Time getSessionTimeout() {\n        return sessionTimeout;\n    }\n\n    public void setSessionTimeout(Time sessionTimeout) {\n        this.sessionTimeout = sessionTimeout;\n    }\n\n    public JoinStream getJoinStream() {\n        return joinStream;\n    }\n\n    public void setJoinStream(JoinStream joinStream) {\n        this.joinStream = joinStream;\n    }\n\n    public enum WindowType {\n        SLIDING_WINDOW,\n        TUMBLING_WINDOW,\n        SESSION_WINDOW\n    }\n\n    public static class JoinStream {\n        private JoinType joinType;\n        private StreamType streamType;\n\n        public JoinStream(JoinType joinType, StreamType streamType) {\n            this.joinType = joinType;\n            this.streamType = streamType;\n        }\n\n        public JoinType getJoinType() {\n            return joinType;\n        }\n\n        public void setJoinType(JoinType joinType) {\n            this.joinType = joinType;\n        }\n\n        public StreamType getStreamType() {\n            return streamType;\n        }\n\n        public void setStreamType(StreamType streamType) {\n            this.streamType = streamType;\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/WindowKey.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window;\n\nimport com.google.common.base.Objects;\nimport org.apache.commons.lang3.StringUtils;\nimport org.apache.rocketmq.streams.core.util.Utils;\n\nimport java.nio.charset.StandardCharsets;\n\npublic class WindowKey {\n    private static final String SPLIT = \"&&\";\n\n    private String operatorName;\n\n    private Long windowStart;\n\n    private Long windowEnd;\n\n    private String key2String;\n\n    public WindowKey(String operatorName, String key2String, Long windowEnd, Long windowStart) {\n        this.operatorName = operatorName;\n        this.windowStart = windowStart;\n        this.windowEnd = windowEnd;\n        this.key2String = key2String;\n    }\n\n    public String getOperatorName() {\n        return operatorName;\n    }\n\n    public void setOperatorName(String operatorName) {\n        this.operatorName = operatorName;\n    }\n\n    public Long getWindowStart() {\n        return windowStart;\n    }\n\n    public void setWindowStart(Long windowStart) {\n        this.windowStart = windowStart;\n    }\n\n    public Long getWindowEnd() {\n        return windowEnd;\n    }\n\n    public void setWindowEnd(Long windowEnd) {\n        this.windowEnd = windowEnd;\n    }\n\n    public String getKey2String() {\n        return key2String;\n    }\n\n    public void setKey2String(String key2String) {\n        this.key2String = key2String;\n    }\n\n\n    public String getKeyAndWindow() {\n        StringBuilder builder = new StringBuilder();\n        builder.append(windowStart)\n                .append(WindowKey.SPLIT)\n                .append(windowEnd)\n                .append(WindowKey.SPLIT)\n                .append(key2String);\n\n        return builder.toString();\n    }\n\n    public static WindowKey byte2WindowKey(byte[] source) {\n        String str = new String(source, StandardCharsets.UTF_8);\n        String[] split = Utils.split(str, WindowKey.SPLIT);\n        return new WindowKey(split[0], split[1],  Long.parseLong(split[2]), Long.parseLong(split[3]));\n    }\n\n\n    public static byte[] windowKey2Byte(WindowKey windowKey) {\n        if (windowKey == null) {\n            return new byte[0];\n        }\n\n        return windowKey.toString().getBytes(StandardCharsets.UTF_8);\n    }\n\n    @Override\n    public int hashCode() {\n        int total = 0;\n        if (!StringUtils.isBlank(operatorName)) {\n            total += operatorName.hashCode();\n        }\n\n        if (!StringUtils.isBlank(key2String)) {\n            total += key2String.hashCode();\n        }\n\n        total += windowEnd;\n\n        total += windowStart;\n\n        return total;\n    }\n\n    @Override\n    public boolean equals(Object o) {\n        if (this == o) return true;\n        if (o == null || getClass() != o.getClass()) return false;\n        WindowKey windowKey = (WindowKey) o;\n        return Objects.equal(operatorName, windowKey.operatorName)\n                && Objects.equal(windowStart, windowKey.windowStart)\n                && Objects.equal(windowEnd, windowKey.windowEnd)\n                && Objects.equal(key2String, windowKey.key2String);\n    }\n\n    @Override\n    public String toString() {\n        StringBuilder builder = new StringBuilder();\n        builder.append(operatorName)\n                .append(WindowKey.SPLIT)\n                .append(key2String)\n                .append(WindowKey.SPLIT)\n                .append(windowEnd)\n                .append(WindowKey.SPLIT)\n                .append(windowStart)\n        ;\n\n        return builder.toString();\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/WindowState.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window;\n\n\nimport com.fasterxml.jackson.core.JsonProcessingException;\nimport io.netty.buffer.ByteBuf;\nimport io.netty.buffer.Unpooled;\nimport org.apache.rocketmq.streams.core.util.Utils;\n\nimport java.io.Serializable;\nimport java.nio.charset.StandardCharsets;\n\n/**\n * windowState data how to encode KV\n * <pre>\n * +-----------+---------------+-----------+-------------+\n * | Int(4)    | Int(4)        | key bytes | value bytes |\n * | key length| value length  |           |             |\n * +-----------+---------------+-----------+-------------+\n * </pre>\n */\npublic class WindowState<K, V> implements Serializable {\n    private static final long serialVersionUID = 1669344441528746814L;\n    private long recordEarliestTimestamp = Long.MAX_VALUE;\n    private long recordLastTimestamp;\n    private K key;\n    private V value;\n    private byte[] keyBytes;\n    private byte[] valueBytes;\n    private Class<?> keyClazz;\n    private Class<?> valueClazz;\n\n    //only for Serializer/Deserializer\n    public WindowState() {\n    }\n\n    public WindowState(K key, V value, long recordLastTimestamp) throws JsonProcessingException {\n        this.key = key;\n        this.value = value;\n        this.recordLastTimestamp = recordLastTimestamp;\n        if (key != null) {\n            this.keyBytes = Utils.object2Byte(key);\n            this.keyClazz = key.getClass();\n        }\n\n        if (value != null) {\n            this.valueBytes = Utils.object2Byte(value);\n            this.valueClazz = value.getClass();\n        }\n    }\n\n    public K getKey() {\n        return key;\n    }\n\n    public void setKey(K key) throws JsonProcessingException {\n        this.key = key;\n        this.keyBytes = Utils.object2Byte(key);\n        if (key != null) {\n            this.keyClazz = key.getClass();\n        }\n    }\n\n    public V getValue() {\n        return value;\n    }\n\n    public void setValue(V value) throws JsonProcessingException {\n        this.value = value;\n        this.valueBytes = Utils.object2Byte(value);\n        if (value != null) {\n            this.valueClazz = value.getClass();\n        }\n    }\n\n    public byte[] getKeyBytes() {\n        return keyBytes;\n    }\n\n    public void setKeyBytes(byte[] keyBytes) {\n        this.keyBytes = keyBytes;\n    }\n\n    public byte[] getValueBytes() {\n        return valueBytes;\n    }\n\n\n    public void setValueBytes(byte[] valueBytes) {\n        this.valueBytes = valueBytes;\n    }\n\n    @SuppressWarnings(\"unchecked\")\n    public Class<V> getValueClazz() {\n        return (Class<V>) valueClazz;\n    }\n\n    public void setValueClazz(Class<?> valueClazz) {\n        this.valueClazz = valueClazz;\n    }\n\n    public void setKeyClazz(Class<?> keyClazz) {\n        this.keyClazz = keyClazz;\n    }\n\n    @SuppressWarnings(\"unchecked\")\n    public Class<K> getKeyClazz() {\n        return (Class<K>) keyClazz;\n    }\n\n    public long getRecordEarliestTimestamp() {\n        return recordEarliestTimestamp;\n    }\n\n    public void setRecordEarliestTimestamp(long recordEarliestTimestamp) {\n        this.recordEarliestTimestamp = recordEarliestTimestamp;\n    }\n\n    public long getRecordLastTimestamp() {\n        return recordLastTimestamp;\n    }\n\n    public void setRecordLastTimestamp(long recordLastTimestamp) {\n        this.recordLastTimestamp = recordLastTimestamp;\n    }\n\n    private static final ByteBuf buf = Unpooled.buffer(16);\n    public static byte[] windowState2Byte(WindowState<?, ?> state) throws Throwable {\n        if (state == null) {\n            return new byte[0];\n        }\n\n        Class<?> keyClazz = state.getKeyClazz();\n        if (keyClazz == null) {\n            keyClazz = state.getKey().getClass();\n        }\n        byte[] keyClazzBytes = keyClazz.getName().getBytes(StandardCharsets.UTF_8);\n\n        byte[] keyBytes = state.getKeyBytes();\n        if (keyBytes == null) {\n            keyBytes = Utils.object2Byte(state.getKey());\n        }\n\n        Class<?> valueClazz = state.getValueClazz();\n        if (valueClazz == null) {\n            valueClazz = state.getValue().getClass();\n        }\n        byte[] valueClazzBytes = valueClazz.getName().getBytes(StandardCharsets.UTF_8);\n\n        byte[] valueBytes = state.getValueBytes();\n        if (valueBytes == null) {\n            valueBytes = Utils.object2Byte(state.getValue());\n        }\n\n\n        int length = 4 + 8 + 8 + 4 + keyClazzBytes.length + 4 + keyBytes.length + 4 + valueClazzBytes.length + 4 + valueBytes.length;\n\n        buf.writeInt(length);\n\n        buf.writeLong(state.getRecordLastTimestamp());\n        buf.writeLong(state.getRecordEarliestTimestamp());\n\n        //key class\n        buf.writeInt(keyClazzBytes.length);\n        buf.writeBytes(keyClazzBytes);\n\n        //key\n        buf.writeInt(keyBytes.length);\n        buf.writeBytes(keyBytes);\n\n        //value class\n        buf.writeInt(valueClazzBytes.length);\n        buf.writeBytes(valueClazzBytes);\n\n        //value\n        buf.writeInt(valueBytes.length);\n        buf.writeBytes(valueBytes);\n\n        byte[] bytes = new byte[buf.readableBytes()];\n        buf.readBytes(bytes);\n\n        buf.clear();\n        return bytes;\n    }\n\n    public static <K,V> WindowState<K,V> byte2WindowState(byte[] bytes) throws Throwable {\n        ByteBuf byteBuf = Unpooled.wrappedBuffer(bytes);\n        int totalLength = byteBuf.readInt();\n        if (bytes.length < totalLength) {\n            //上层已经拆好了包\n            throw new IllegalArgumentException(\"byteBuf length less than total\");\n        }\n\n        long recordLastTimestamp = byteBuf.readLong();\n        long recordEarliestTimestamp = byteBuf.readLong();\n\n        //key class\n        int keyClazzLength = byteBuf.readInt();\n        ByteBuf buf = byteBuf.readBytes(keyClazzLength);\n        byte[] keyClazzBytes = new byte[keyClazzLength];\n        buf.readBytes(keyClazzBytes);\n        //实例化\n        String keyClassName = new String(keyClazzBytes, StandardCharsets.UTF_8);\n        Class<?> keyClazz = Class.forName(keyClassName);\n\n        //key\n        int keyLength = byteBuf.readInt();\n        ByteBuf keyBuf = byteBuf.readBytes(keyLength);\n        byte[] keyBytes = new byte[keyLength];\n        keyBuf.readBytes(keyBytes);\n\n        //value class\n        int valueClazzLength = byteBuf.readInt();\n        ByteBuf valueClazzBuf = byteBuf.readBytes(valueClazzLength);\n        byte[] valueClazzBytes = new byte[valueClazzLength];\n        valueClazzBuf.readBytes(valueClazzBytes);\n        //实例化\n        String valueClassName = new String(valueClazzBytes, StandardCharsets.UTF_8);\n        Class<?> valueClazz = Class.forName(valueClassName);\n\n        //value\n        int valueLength = byteBuf.readInt();\n        ByteBuf valueBuf = byteBuf.readBytes(valueLength);\n        byte[] valueBytes = new byte[valueLength];\n        valueBuf.readBytes(valueBytes);\n\n        WindowState<K, V> result = new WindowState<>();\n        result.setRecordLastTimestamp(recordLastTimestamp);\n        result.setRecordEarliestTimestamp(recordEarliestTimestamp);\n        result.setKeyBytes(keyBytes);\n        result.setValueBytes(valueBytes);\n        result.setKeyClazz(keyClazz);\n        result.setValueClazz(valueClazz);\n        result.setKey(Utils.byte2Object(keyBytes, result.getKeyClazz()));\n        result.setValue(Utils.byte2Object(valueBytes, result.getValueClazz()));\n\n        byteBuf.release();\n        buf.release();\n        keyBuf.release();\n        valueBuf.release();\n        valueClazzBuf.release();\n        return result;\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/WindowStore.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window;\n\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.state.StateStore;\nimport org.apache.rocketmq.streams.core.util.ColumnFamilyUtil;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.List;\n\npublic class WindowStore<K, V> {\n    private static final Logger logger = LoggerFactory.getLogger(WindowStore.class.getName());\n\n    private StateStore stateStore;\n    private ValueMapperAction<byte[], WindowState<K, V>> bytes2State;\n    private ValueMapperAction<WindowState<K, V>, byte[]> state2Bytes;\n\n\n    public WindowStore(StateStore stateStore,\n                       ValueMapperAction<byte[], WindowState<K, V>> bytes2State,\n                       ValueMapperAction<WindowState<K, V>, byte[]> state2Bytes) {\n        this.stateStore = stateStore;\n        this.bytes2State = bytes2State;\n        this.state2Bytes = state2Bytes;\n    }\n\n    public void put(MessageQueue stateTopicMessageQueue, WindowKey windowKey, WindowState<K, V> value) throws Throwable {\n        logger.debug(\"put key into store, key: \" + windowKey);\n        byte[] keyBytes = WindowKey.windowKey2Byte(windowKey);\n        byte[] valueBytes = this.state2Bytes.convert(value);\n\n        this.stateStore.put(stateTopicMessageQueue, ColumnFamilyUtil.WINDOW_STATE_CF, keyBytes, valueBytes);\n    }\n\n    public WindowState<K, V> get(WindowKey windowKey) throws Throwable {\n        byte[] bytes = WindowKey.windowKey2Byte(windowKey);\n        byte[] valueBytes = this.stateStore.get(ColumnFamilyUtil.WINDOW_STATE_CF, bytes);\n        return deserializerState(valueBytes);\n    }\n\n    public List<Pair<WindowKey, WindowState<K, V>>> searchLessThanWatermark(String operatorName, long lessThanThisTime) throws Throwable {\n        List<Pair<byte[], byte[]>> windowStateBytes = this.stateStore.searchStateLessThanWatermark(operatorName, lessThanThisTime, WindowKey::byte2WindowKey);\n        return deserializerState(windowStateBytes);\n    }\n\n    public List<Pair<WindowKey, WindowState<K, V>>> searchMatchKeyPrefix(String operatorName) throws Throwable {\n        List<Pair<byte[], byte[]>> pairs = this.stateStore.searchStateLessThanWatermark(operatorName, Long.MAX_VALUE, WindowKey::byte2WindowKey);\n\n        return deserializerState(pairs);\n    }\n\n    public void deleteByKey(WindowKey windowKey) throws Throwable {\n        if (windowKey == null) {\n            return;\n        }\n        byte[] keyBytes = WindowKey.windowKey2Byte(windowKey);\n        this.stateStore.delete(keyBytes);\n    }\n\n    private List<Pair<WindowKey, WindowState<K, V>>> deserializerState(List<Pair<byte[], byte[]>> windowStateBytes) throws Throwable {\n        List<Pair<WindowKey, WindowState<K, V>>> result = new ArrayList<>();\n        if (windowStateBytes == null || windowStateBytes.size() == 0) {\n            return result;\n        }\n\n\n        for (Pair<byte[], byte[]> pair : windowStateBytes) {\n            byte[] keyBytes = pair.getKey();\n            WindowKey key = WindowKey.byte2WindowKey(keyBytes);\n            WindowState<K, V> state = this.deserializerState(pair.getValue());\n\n            Pair<WindowKey, WindowState<K, V>> temp = new Pair<>(key, state);\n            result.add(temp);\n        }\n        return result;\n    }\n\n    private WindowState<K, V> deserializerState(byte[] source) throws Throwable {\n        if (source == null) {\n            return null;\n        }\n\n        return this.bytes2State.convert(source);\n    }\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AbstractWindowFire.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window.fire;\n\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.state.StateStore;\nimport org.apache.rocketmq.streams.core.util.Utils;\n\nimport java.util.HashSet;\nimport java.util.Set;\nimport java.util.function.BiFunction;\n\npublic abstract class AbstractWindowFire<K, V> implements WindowFire<K, V> {\n    protected final StreamContext<V> context;\n    private final MessageQueue stateTopicMessageQueue;\n    private final BiFunction<Long, MessageQueue, Long> commitWatermark;\n\n    public AbstractWindowFire(StreamContext<V> context,\n                              MessageQueue stateTopicMessageQueue,\n                              BiFunction<Long, MessageQueue, Long> commitWatermark) {\n        this.context = context;\n        this.stateTopicMessageQueue = stateTopicMessageQueue;\n        this.commitWatermark = commitWatermark;\n    }\n\n    void commitWatermark(long watermark) {\n        this.commitWatermark.apply(watermark, stateTopicMessageQueue);\n    }\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AccumulatorSessionWindowFire.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window.fire;\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RStreamsException;\nimport org.apache.rocketmq.streams.core.function.accumulator.Accumulator;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.window.WindowState;\nimport org.apache.rocketmq.streams.core.window.WindowStore;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.Iterator;\nimport java.util.List;\nimport java.util.Properties;\nimport java.util.function.BiFunction;\n\n\npublic class AccumulatorSessionWindowFire<K, R, V, OV> extends AbstractWindowFire<K, V> {\n    private static final Logger logger = LoggerFactory.getLogger(AccumulatorSessionWindowFire.class);\n\n    private final WindowStore<K, Accumulator<R, OV>> windowStore;\n\n    public AccumulatorSessionWindowFire(WindowStore<K, Accumulator<R, OV>> windowStore,\n                                        StreamContext<V> context,\n                                        MessageQueue stateTopicMessageQueue,\n                                        BiFunction<Long, MessageQueue, Long> commitWatermark) {\n        super(context, stateTopicMessageQueue, commitWatermark);\n        this.windowStore = windowStore;\n    }\n\n    public List<WindowKey> fire(String operatorName, long watermark) {\n        List<WindowKey> fired = new ArrayList<>();\n\n        try {\n            List<Pair<WindowKey, WindowState<K, Accumulator<R, OV>>>> pairs = windowStore.searchLessThanWatermark(operatorName, watermark);\n\n            Iterator<Pair<WindowKey, WindowState<K, Accumulator<R, OV>>>> iterator = pairs.iterator();\n            while (iterator.hasNext()) {\n                Pair<WindowKey, WindowState<K, Accumulator<R, OV>>> pair = iterator.next();\n                WindowKey windowKey = pair.getKey();\n                WindowState<K, Accumulator<R, OV>> state = pair.getValue();\n\n                long windowEnd = windowKey.getWindowEnd();\n                long windowBegin;\n                if (state.getRecordEarliestTimestamp() == Long.MAX_VALUE) {\n                    windowBegin = windowKey.getWindowStart();\n                } else {\n                    windowBegin = state.getRecordEarliestTimestamp();\n                }\n\n                logger.info(\"fire session,windowKey={}, search keyPrefix={}, window: [{} - {}]\",\n                        windowKey, state.getKey().toString(), Utils.format(windowBegin), Utils.format(windowEnd));\n\n                Properties header = context.getHeader();\n                header.put(Constant.WINDOW_START_TIME, windowBegin);\n                header.put(Constant.WINDOW_END_TIME, windowEnd);\n\n                Accumulator<R, OV> value = state.getValue();\n                OV data = value.result(header);\n\n                Data<K, OV> result = new Data<>(state.getKey(), data, state.getRecordLastTimestamp(), header);\n                Data<K, V> convert = this.convert(result);\n\n                context.forward(convert);\n\n                //删除状态\n                windowStore.deleteByKey(windowKey);\n\n                fired.add(windowKey);\n            }\n\n            return fired;\n        } catch (Throwable t) {\n            String format = String.format(\"fire session window error, name:%s\", operatorName);\n            throw new RStreamsException(format, t);\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AccumulatorWindowFire.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window.fire;\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RStreamsException;\nimport org.apache.rocketmq.streams.core.function.accumulator.Accumulator;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.window.WindowState;\nimport org.apache.rocketmq.streams.core.window.WindowStore;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.List;\nimport java.util.Properties;\nimport java.util.function.BiFunction;\n\npublic class AccumulatorWindowFire<K, R, V, OV> extends AbstractWindowFire<K, V> {\n    private static final Logger logger = LoggerFactory.getLogger(AccumulatorWindowFire.class);\n\n    private final WindowStore<K, Accumulator<R, OV>> windowStore;\n\n    public AccumulatorWindowFire(WindowStore<K, Accumulator<R, OV>> windowStore,\n                                 StreamContext<V> context,\n                                 MessageQueue stateTopicMessageQueue,\n                                 BiFunction<Long, MessageQueue, Long> commitWatermark) {\n        super(context, stateTopicMessageQueue, commitWatermark);\n        this.windowStore = windowStore;\n    }\n\n\n    public List<WindowKey> fire(String operatorName, long watermark) {\n        List<WindowKey> fired = new ArrayList<>();\n\n        try {\n            List<Pair<WindowKey, WindowState<K, Accumulator<R, OV>>>> pairs = windowStore.searchLessThanWatermark(operatorName, watermark);\n\n            //pairs中最后一个时间最小，应该最先触发\n            for (int i = pairs.size() - 1; i >= 0; i--) {\n                Pair<WindowKey, WindowState<K, Accumulator<R, OV>>> pair = pairs.get(i);\n\n                WindowKey windowKey = pair.getKey();\n                WindowState<K, Accumulator<R, OV>> value = pair.getValue();\n\n                Long windowEnd = windowKey.getWindowEnd();\n\n                Properties header = context.getHeader();\n                header.put(Constant.WINDOW_START_TIME, windowKey.getWindowStart());\n                header.put(Constant.WINDOW_END_TIME, windowEnd);\n\n                Accumulator<R, OV> rovAccumulator = value.getValue();\n                OV data = rovAccumulator.result(header);\n\n                Data<K, OV> result = new Data<>(value.getKey(), data, value.getRecordLastTimestamp(), header);\n                Data<K, V> convert = this.convert(result);\n\n                if (logger.isDebugEnabled()) {\n                    logger.debug(\"fire window, windowKey={}, search watermark={}, window: [{} - {}], data to next:[{}]\", windowKey,\n                            watermark, Utils.format(windowKey.getWindowStart()), Utils.format(windowEnd), convert);\n                }\n\n                context.forward(convert);\n\n                //删除状态\n                windowStore.deleteByKey(windowKey);\n\n                fired.add(windowKey);\n            }\n\n        } catch (Throwable t) {\n            String format = String.format(\"fire window error, watermark:%s, operatorName:%s\", watermark, operatorName);\n            throw new RStreamsException(format, t);\n        }\n        return fired;\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AggregateSessionWindowFire.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window.fire;\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RStreamsException;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.window.WindowState;\nimport org.apache.rocketmq.streams.core.window.WindowStore;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.Iterator;\nimport java.util.List;\nimport java.util.Properties;\nimport java.util.function.BiFunction;\n\npublic class AggregateSessionWindowFire<K, V, OV> extends AbstractWindowFire<K, V> {\n    private static final Logger logger = LoggerFactory.getLogger(AggregateWindowFire.class);\n\n    private final WindowStore<K, OV> windowStore;\n\n    public AggregateSessionWindowFire(WindowStore<K, OV> windowStore,\n                                      MessageQueue stateTopicMessageQueue,\n                                      StreamContext<V> context,\n                                      BiFunction<Long, MessageQueue, Long> commitWatermark) {\n        super(context, stateTopicMessageQueue, commitWatermark);\n        this.windowStore = windowStore;\n    }\n\n    @Override\n    public List<WindowKey> fire(String operatorName, long watermark) {\n        List<WindowKey> fired = new ArrayList<>();\n\n        try {\n            List<Pair<WindowKey, WindowState<K, OV>>> pairs = this.windowStore.searchLessThanWatermark(operatorName, watermark);\n\n            Iterator<Pair<WindowKey, WindowState<K, OV>>> iterator = pairs.iterator();\n            while (iterator.hasNext()) {\n                Pair<WindowKey, WindowState<K, OV>> pair = iterator.next();\n                WindowKey windowKey = pair.getKey();\n                WindowState<K, OV> state = pair.getValue();\n\n                long windowEnd = windowKey.getWindowEnd();\n                long windowBegin;\n                if (state.getRecordEarliestTimestamp() == Long.MAX_VALUE) {\n                    windowBegin = windowKey.getWindowStart();\n                } else {\n                    windowBegin = state.getRecordEarliestTimestamp();\n                }\n\n                logger.info(\"fire session,windowKey={}, search keyPrefix={}, window: [{} - {}]\", windowKey,\n                        state.getKey().toString(), Utils.format(windowBegin), Utils.format(windowEnd));\n\n                Properties header = this.context.getHeader();\n                header.put(Constant.WINDOW_START_TIME, windowBegin);\n                header.put(Constant.WINDOW_END_TIME, windowEnd);\n\n                Data<K, OV> result = new Data<>(state.getKey(), state.getValue(), state.getRecordLastTimestamp(), header);\n                Data<K, V> convert = this.convert(result);\n\n                this.context.forward(convert);\n\n                //删除状态\n                this.windowStore.deleteByKey(windowKey);\n\n                fired.add(windowKey);\n            }\n\n            return fired;\n        } catch (Throwable t) {\n            String format = String.format(\"fire session window error, name:%s\", operatorName);\n            throw new RStreamsException(format, t);\n        }\n    }\n\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/AggregateWindowFire.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window.fire;\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RStreamsException;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.window.WindowState;\nimport org.apache.rocketmq.streams.core.window.WindowStore;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.List;\nimport java.util.Properties;\nimport java.util.function.BiFunction;\n\npublic class AggregateWindowFire<K, V, OV> extends AbstractWindowFire<K, V> {\n    private static final Logger logger = LoggerFactory.getLogger(AggregateWindowFire.class);\n\n    private final WindowStore<K, OV> windowStore;\n\n    public AggregateWindowFire(WindowStore<K, OV> windowStore,\n                               MessageQueue stateTopicMessageQueue,\n                               StreamContext<V> context,\n                               BiFunction<Long, MessageQueue, Long> commitWatermark) {\n        super(context, stateTopicMessageQueue, commitWatermark);\n        this.windowStore = windowStore;\n    }\n\n    @Override\n    public List<WindowKey> fire(String operatorName, long watermark) {\n        List<WindowKey> fired = new ArrayList<>();\n\n        try {\n            List<Pair<WindowKey, WindowState<K, OV>>> pairs = this.windowStore.searchLessThanWatermark(operatorName, watermark);\n\n            //pairs中最后一个时间最小，应该最先触发\n            for (int i = pairs.size() - 1; i >= 0; i--) {\n                Pair<WindowKey, WindowState<K, OV>> pair = pairs.get(i);\n\n                WindowKey windowKey = pair.getKey();\n                WindowState<K, OV> value = pair.getValue();\n\n                Long windowEnd = windowKey.getWindowEnd();\n\n                Properties header = this.context.getHeader();\n                header.put(Constant.WINDOW_START_TIME, windowKey.getWindowStart());\n                header.put(Constant.WINDOW_END_TIME, windowEnd);\n                Data<K, OV> result = new Data<>(value.getKey(), value.getValue(), value.getRecordLastTimestamp(), header);\n                Data<K, V> convert = this.convert(result);\n\n                if (logger.isDebugEnabled()) {\n                    logger.debug(\"fire window, windowKey={}, search watermark={}, window: [{} - {}], data to next:[{}]\", windowKey,\n                            watermark, Utils.format(windowKey.getWindowStart()), Utils.format(windowEnd), convert);\n                }\n\n                this.context.forward(convert);\n\n                //删除状态\n                this.windowStore.deleteByKey(windowKey);\n\n                fired.add(windowKey);\n            }\n\n            return fired;\n        } catch (Throwable t) {\n            String format = String.format(\"fire window error, watermark:%s, operatorName:%s\", watermark, operatorName);\n            throw new RStreamsException(format, t);\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/IdleWindowScaner.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window.fire;\n\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.window.StreamType;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.Iterator;\nimport java.util.Map;\nimport java.util.concurrent.ConcurrentHashMap;\nimport java.util.concurrent.ScheduledExecutorService;\nimport java.util.concurrent.TimeUnit;\n\n\npublic class IdleWindowScaner implements AutoCloseable {\n    private static final Logger logger = LoggerFactory.getLogger(IdleWindowScaner.class.getName());\n\n    private final Integer maxIdleTime;\n    private final ScheduledExecutorService executor;\n\n    private final ConcurrentHashMap<WindowKey, TimeType> lastUpdateTime2WindowKey = new ConcurrentHashMap<>(16);\n\n    private final ConcurrentHashMap<WindowKey, AccumulatorWindowFire<?, ?, ?, ?>> fireWindowCallBack = new ConcurrentHashMap<>(16);\n    private final ConcurrentHashMap<WindowKey, AccumulatorSessionWindowFire<?, ?, ?, ?>> fireSessionWindowCallback = new ConcurrentHashMap<>(16);\n    private final ConcurrentHashMap<WindowKey, AggregateWindowFire<?, ?, ?>> windowKeyAggregate = new ConcurrentHashMap<>(16);\n    private final ConcurrentHashMap<WindowKey, AggregateSessionWindowFire<?, ?, ?>> windowKeyAggregateSession = new ConcurrentHashMap<>(16);\n\n    private final ConcurrentHashMap<WindowKey, JoinWindowFire<?, ?, ?, ?>> fireJoinWindowCallback = new ConcurrentHashMap<>(16);\n\n\n    public IdleWindowScaner(Integer maxIdleTime, ScheduledExecutorService executor) {\n        this.maxIdleTime = maxIdleTime;\n        this.executor = executor;\n        this.executor.scheduleAtFixedRate(() -> {\n            try {\n                scanAndFireWindow();\n            } catch (Throwable t) {\n                logger.error(\"scan and fire the idle window error.\", t);\n            }\n        }, 0, 1000, TimeUnit.MILLISECONDS);\n    }\n\n    public void putAccumulatorWindowCallback(WindowKey windowKey, long watermark, AccumulatorWindowFire<?, ?, ?, ?> function) {\n        this.fireWindowCallBack.putIfAbsent(windowKey, function);\n        this.lastUpdateTime2WindowKey.compute(windowKey, (key, timeType) -> {\n            if (timeType == null) {\n                timeType = new TimeType(Type.AccumulatorWindow, System.currentTimeMillis(), watermark);\n            } else {\n                timeType.setUpdateTime(System.currentTimeMillis());\n                timeType.setWatermark(watermark);\n            }\n            return timeType;\n        });\n    }\n\n    public void putAccumulatorSessionWindowCallback(WindowKey windowKey, long watermark, AccumulatorSessionWindowFire<?, ?, ?, ?> function) {\n        this.fireSessionWindowCallback.putIfAbsent(windowKey, function);\n        this.lastUpdateTime2WindowKey.compute(windowKey, (key, timeType) -> {\n            if (timeType == null) {\n                timeType = new TimeType(Type.AccumulatorSessionWindow, System.currentTimeMillis(), watermark);\n            } else {\n                timeType.setUpdateTime(System.currentTimeMillis());\n                timeType.setWatermark(watermark);\n            }\n            return timeType;\n        });\n    }\n\n    public void putAggregateWindowCallback(WindowKey windowKey, long watermark, AggregateWindowFire<?, ?, ?> function) {\n        this.windowKeyAggregate.putIfAbsent(windowKey, function);\n        this.lastUpdateTime2WindowKey.compute(windowKey, (key, timeType) -> {\n            if (timeType == null) {\n                timeType = new TimeType(Type.AggregateWindow, System.currentTimeMillis(), watermark);\n            } else {\n                timeType.setUpdateTime(System.currentTimeMillis());\n                timeType.setWatermark(watermark);\n            }\n            return timeType;\n        });\n    }\n\n    public void putAggregateSessionWindowCallback(WindowKey windowKey, long watermark, AggregateSessionWindowFire<?, ?, ?> function) {\n        this.windowKeyAggregateSession.putIfAbsent(windowKey, function);\n        this.lastUpdateTime2WindowKey.compute(windowKey, (key, timeType) -> {\n            if (timeType == null) {\n                timeType = new TimeType(Type.AggregateSessionWindow, System.currentTimeMillis(), watermark);\n            } else {\n                timeType.setUpdateTime(System.currentTimeMillis());\n                timeType.setWatermark(watermark);\n            }\n            return timeType;\n        });\n    }\n\n    public void putJoinWindowCallback(WindowKey windowKey, long watermark, JoinWindowFire<?, ?, ?, ?> function) {\n        this.fireJoinWindowCallback.putIfAbsent(windowKey, function);\n        this.lastUpdateTime2WindowKey.compute(windowKey, (key, timeType) -> {\n            if (timeType == null) {\n                timeType = new TimeType(Type.JoinWindow, System.currentTimeMillis(), watermark);\n            } else {\n                timeType.setUpdateTime(System.currentTimeMillis());\n                timeType.setWatermark(watermark);\n            }\n            return timeType;\n        });\n    }\n\n    public void removeOldAccumulatorSession(WindowKey oldWindowKey) {\n        if (oldWindowKey == null) {\n            return;\n        }\n\n        TimeType timeType = this.lastUpdateTime2WindowKey.get(oldWindowKey);\n        if (timeType != null && timeType.getType() == Type.AccumulatorSessionWindow) {\n            this.lastUpdateTime2WindowKey.remove(oldWindowKey);\n        }\n        this.fireSessionWindowCallback.remove(oldWindowKey);\n    }\n\n    public void removeOldAggregateSession(WindowKey oldWindowKey) {\n        TimeType timeType = this.lastUpdateTime2WindowKey.get(oldWindowKey);\n        if (timeType != null\n                && timeType.getType() == Type.AggregateSessionWindow) {\n            this.lastUpdateTime2WindowKey.remove(oldWindowKey);\n        }\n        this.windowKeyAggregateSession.remove(oldWindowKey);\n    }\n\n    public void removeWindowKey(WindowKey windowKey) {\n        lastUpdateTime2WindowKey.remove(windowKey);\n\n        fireWindowCallBack.remove(windowKey);\n        fireSessionWindowCallback.remove(windowKey);\n\n        windowKeyAggregate.remove(windowKey);\n        windowKeyAggregateSession.remove(windowKey);\n\n        fireJoinWindowCallback.remove(windowKey);\n    }\n\n    private void scanAndFireWindow() throws Throwable {\n        Iterator<Map.Entry<WindowKey, TimeType>> iterator = this.lastUpdateTime2WindowKey.entrySet().iterator();\n        while (iterator.hasNext()) {\n            Map.Entry<WindowKey, TimeType> next = iterator.next();\n\n            WindowKey windowKey = next.getKey();\n            TimeType timeType = next.getValue();\n\n            Type type = timeType.getType();\n            long updateTime = timeType.getUpdateTime();\n\n            long idleTime = System.currentTimeMillis() - updateTime;\n\n            switch (type) {\n                case AggregateSessionWindow:\n                case AccumulatorSessionWindow: {\n                    long watermark = timeType.getWatermark() + idleTime;\n                    if (watermark > windowKey.getWindowEnd()) {\n                        try {\n                            doFire(windowKey, type, watermark);\n                        } finally {\n                            iterator.remove();\n                        }\n                    }\n                    break;\n                }\n                case AccumulatorWindow:\n                case JoinWindow:\n                case AggregateWindow: {\n                    long watermark = timeType.getWatermark() + idleTime;\n                    if (idleTime > this.maxIdleTime && watermark > windowKey.getWindowEnd()) {\n                        try {\n                            doFire(windowKey, type, watermark);\n                        } finally {\n                            iterator.remove();\n                        }\n                    }\n                    break;\n                }\n                default:\n                    throw new UnsupportedOperationException(\"unknown window type: \" + type);\n            }\n        }\n    }\n\n    private void doFire(WindowKey windowKey, Type type, long watermark) throws Throwable {\n        String operatorName = windowKey.getOperatorName();\n\n        switch (type) {\n            case AccumulatorWindow: {\n                AccumulatorWindowFire<?, ?, ?, ?> func = this.fireWindowCallBack.remove(windowKey);\n                if (func != null) {\n                    //write the result out, delete the state from local and remote\n                    func.fire(operatorName, watermark);\n                    //commit watermark to local and remote.\n                    func.commitWatermark(watermark);\n                }\n                break;\n            }\n            case AccumulatorSessionWindow: {\n                AccumulatorSessionWindowFire<?, ?, ?, ?> accumulatorSessionWindowFire = this.fireSessionWindowCallback.remove(windowKey);\n                if (accumulatorSessionWindowFire != null) {\n                    accumulatorSessionWindowFire.fire(operatorName, watermark);\n                    accumulatorSessionWindowFire.commitWatermark(watermark);\n                }\n                break;\n            }\n            case AggregateWindow: {\n                AggregateWindowFire<?, ?, ?> aggregateWindowFire = this.windowKeyAggregate.remove(windowKey);\n                if (aggregateWindowFire != null) {\n                    aggregateWindowFire.fire(operatorName, watermark);\n                    aggregateWindowFire.commitWatermark(watermark);\n                }\n                break;\n            }\n            case AggregateSessionWindow: {\n                AggregateSessionWindowFire<?, ?, ?> sessionWindowFire = this.windowKeyAggregateSession.remove(windowKey);\n                if (sessionWindowFire != null) {\n                    sessionWindowFire.fire(operatorName, watermark);\n                    sessionWindowFire.commitWatermark(watermark);\n                }\n                break;\n            }\n            case JoinWindow: {\n                JoinWindowFire<?, ?, ?, ?> joinWindowFire = this.fireJoinWindowCallback.remove(windowKey);\n                if (joinWindowFire != null) {\n                    String name = operatorName.substring(0, operatorName.lastIndexOf(Constant.SPLIT));\n                    String streamType = operatorName.substring(operatorName.lastIndexOf(Constant.SPLIT) + 1);\n\n                    joinWindowFire.fire(name, watermark, StreamType.valueOf(streamType));\n                    joinWindowFire.commitWatermark(watermark);\n                }\n                break;\n            }\n        }\n    }\n\n    @Override\n    public void close() throws Exception {\n        this.executor.shutdown();\n    }\n\n    static class TimeType {\n        private Type type;\n        private long updateTime;\n        private long watermark;\n\n        public TimeType(Type type, long updateTime, long watermark) {\n            this.type = type;\n            this.updateTime = updateTime;\n            this.watermark = watermark;\n        }\n\n        public Type getType() {\n            return type;\n        }\n\n        public void setType(Type type) {\n            this.type = type;\n        }\n\n        public long getUpdateTime() {\n            return updateTime;\n        }\n\n        public void setUpdateTime(long updateTime) {\n            this.updateTime = updateTime;\n        }\n\n        public long getWatermark() {\n            return watermark;\n        }\n\n        public void setWatermark(long watermark) {\n            this.watermark = watermark;\n        }\n    }\n\n    enum Type {\n        AccumulatorWindow, AccumulatorSessionWindow, AggregateWindow, AggregateSessionWindow, JoinWindow\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/JoinWindowFire.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window.fire;\n\nimport org.apache.rocketmq.common.message.MessageQueue;\nimport org.apache.rocketmq.streams.core.common.Constant;\nimport org.apache.rocketmq.streams.core.exception.RStreamsException;\nimport org.apache.rocketmq.streams.core.function.ValueJoinAction;\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.running.StreamContext;\nimport org.apache.rocketmq.streams.core.state.StateStore;\nimport org.apache.rocketmq.streams.core.util.ColumnFamilyUtil;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.Utils;\nimport org.apache.rocketmq.streams.core.window.JoinType;\nimport org.apache.rocketmq.streams.core.window.StreamType;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.window.WindowState;\nimport org.apache.rocketmq.streams.core.window.WindowStore;\nimport org.slf4j.Logger;\nimport org.slf4j.LoggerFactory;\n\nimport java.util.ArrayList;\nimport java.util.Comparator;\nimport java.util.HashSet;\nimport java.util.List;\nimport java.util.Properties;\nimport java.util.Set;\nimport java.util.function.BiFunction;\n\npublic class JoinWindowFire<K, V1, V2, OUT> {\n    private static final Logger logger = LoggerFactory.getLogger(JoinWindowFire.class);\n\n    private final JoinType joinType;\n    private final MessageQueue stateTopicMessageQueue;\n    private final StreamContext<Object> context;\n    private final ValueJoinAction<V1, V2, OUT> joinAction;\n    private final WindowStore<K, V1> leftWindowStore;\n    private final WindowStore<K, V2> rightWindowStore;\n    private final BiFunction<Long, MessageQueue, Long> commitWatermark;\n\n    public JoinWindowFire(JoinType joinType,\n                          MessageQueue stateTopicMessageQueue,\n                          StreamContext<Object> context,\n                          ValueJoinAction<V1, V2, OUT> joinAction,\n                          WindowStore<K, V1> leftWindowStore,\n                          WindowStore<K, V2> rightWindowStore,\n                          BiFunction<Long, MessageQueue, Long> commitWatermark) {\n        this.joinType = joinType;\n        this.stateTopicMessageQueue = stateTopicMessageQueue;\n        this.context = context;\n        this.joinAction = joinAction;\n        this.leftWindowStore = leftWindowStore;\n        this.rightWindowStore = rightWindowStore;\n        this.commitWatermark = commitWatermark;\n    }\n\n    public List<WindowKey> fire(String operatorName, long watermark, StreamType streamType) {\n        List<WindowKey> fired = new ArrayList<>();\n\n        try {\n            String leftWindow = Utils.buildKey(operatorName, StreamType.LEFT_STREAM.name());\n            List<Pair<WindowKey, WindowState<K, V1>>> leftPairs = this.leftWindowStore.searchLessThanWatermark(leftWindow, watermark);\n            if (leftPairs.size() != 0) {\n                for (Pair<WindowKey, WindowState<K, V1>> leftPair : leftPairs) {\n                    logger.debug(\"search with key prefix:{} and watermark:{}, find window: {}\", leftWindow, Utils.format(watermark), leftPair.getKey());\n                }\n            }\n\n            String rightWindow = Utils.buildKey(operatorName, StreamType.RIGHT_STREAM.name());\n            List<Pair<WindowKey, WindowState<K, V2>>> rightPairs = this.rightWindowStore.searchLessThanWatermark(rightWindow, watermark);\n            if (rightPairs.size() != 0) {\n                for (Pair<WindowKey, WindowState<K, V2>> rightPair : rightPairs) {\n                    logger.debug(\"search with key prefix:{} and watermark:{}, find window: {}\", rightWindow, Utils.format(watermark), rightPair.getKey());\n                }\n            }\n\n            if (leftPairs.size() == 0 && rightPairs.size() == 0) {\n                logger.debug(\"left window and right window are all empty, watermark:{}.\" +\n                        \"left window operatorName:{}, right window operatorName:{}\", Utils.format(watermark), leftWindow, rightWindow);\n                return fired;\n            }\n\n            leftPairs.sort(Comparator.comparing(pair -> {\n                WindowKey key = pair.getKey();\n                return key.getWindowEnd();\n            }));\n            rightPairs.sort(Comparator.comparing(pair -> {\n                WindowKey key = pair.getKey();\n                return key.getWindowEnd();\n            }));\n\n            switch (joinType) {\n                case INNER_JOIN:\n                    //匹配上才触发\n                    for (Pair<WindowKey, WindowState<K, V1>> leftPair : leftPairs) {\n                        WindowKey leftWindowKey = leftPair.getKey();\n                        String leftPrefix = leftWindowKey.getKeyAndWindow();\n\n                        for (Pair<WindowKey, WindowState<K, V2>> rightPair : rightPairs) {\n                            String rightPrefix = rightPair.getKey().getKeyAndWindow();\n\n                            //相同window中相同key，聚合\n                            if (leftPrefix.equals(rightPrefix)) {\n                                //do fire\n                                V1 o1 = leftPair.getValue().getValue();\n                                V2 o2 = rightPair.getValue().getValue();\n\n                                OUT out = this.joinAction.apply(o1, o2);\n\n                                Properties header = this.context.getHeader();\n                                header.put(Constant.WINDOW_START_TIME, leftWindowKey.getWindowStart());\n                                header.put(Constant.WINDOW_END_TIME, leftWindowKey.getWindowEnd());\n\n                                assert leftPair.getValue().getKey() == rightPair.getValue().getKey();\n                                Data<K, OUT> result = new Data<>(leftPair.getValue().getKey(), out, this.context.getDataTime(), header);\n                                Data<K, Object> convert = this.convert(result);\n\n                                this.context.forward(convert);\n\n                                fired.add(leftWindowKey);\n                            }\n                        }\n                    }\n                    break;\n                case LEFT_JOIN:\n                    switch (streamType) {\n                        case LEFT_STREAM:\n                            //左流全部触发，不管右流匹配上没\n                            for (Pair<WindowKey, WindowState<K, V1>> leftPair : leftPairs) {\n                                WindowKey leftWindowKey = leftPair.getKey();\n\n                                fired.add(leftWindowKey);\n\n                                String leftPrefix = leftWindowKey.getKeyAndWindow();\n                                Pair<WindowKey, WindowState<K, V2>> targetPair = null;\n\n                                for (Pair<WindowKey, WindowState<K, V2>> rightPair : rightPairs) {\n                                    if (rightPair.getKey().getKeyAndWindow().equals(leftPrefix)) {\n                                        targetPair = rightPair;\n                                        break;\n                                    }\n                                }\n\n                                //fire\n                                V1 o1 = leftPair.getValue().getValue();\n                                V2 o2 = null;\n                                if (targetPair != null) {\n                                    o2 = targetPair.getValue().getValue();\n                                    fired.add(targetPair.getKey());\n\n                                    assert leftPair.getValue().getKey() == targetPair.getValue().getKey();\n                                }\n\n                                OUT out = this.joinAction.apply(o1, o2);\n                                Properties header = this.context.getHeader();\n                                header.put(Constant.WINDOW_START_TIME, leftWindowKey.getWindowStart());\n                                header.put(Constant.WINDOW_END_TIME, leftWindowKey.getWindowEnd());\n\n\n                                Data<K, OUT> result = new Data<>(leftPair.getValue().getKey(), out, this.context.getDataTime(), header);\n                                Data<K, Object> convert = this.convert(result);\n\n                                this.context.forward(convert);\n                            }\n                            break;\n                        case RIGHT_STREAM:\n                            //do nothing.\n                    }\n                    break;\n            }\n\n            if (leftPairs.size() != 0) {\n                logger.debug(\"delete left window.\");\n                for (Pair<WindowKey, WindowState<K, V1>> leftPair : leftPairs) {\n                    this.leftWindowStore.deleteByKey(leftPair.getKey());\n                }\n            }\n\n            if (rightPairs.size() != 0) {\n                logger.debug(\"delete right window.\");\n                for (Pair<WindowKey, WindowState<K, V2>> rightPair : rightPairs) {\n                    this.rightWindowStore.deleteByKey(rightPair.getKey());\n                }\n            }\n        } catch (Throwable t) {\n            String format = String.format(\"fire window error, watermark:%s.\", watermark);\n            throw new RStreamsException(format, t);\n        }\n\n        return fired;\n    }\n\n    @SuppressWarnings(\"unchecked\")\n    private <K> Data<K, Object> convert(Data<?, ?> data) {\n        return (Data<K, Object>) new Data<>(data.getKey(), data.getValue(), data.getTimestamp(), data.getHeader());\n    }\n\n    void commitWatermark(long watermark) throws Throwable {\n        StateStore stateStore = this.context.getStateStore();\n\n        //get old watermark\n        byte[] keyBytes = Utils.watermarkKeyBytes(stateTopicMessageQueue, Constant.WATERMARK_KEY);\n        byte[] watermarkBytes = stateStore.get(ColumnFamilyUtil.WATERMARK_STATE_CF, keyBytes);\n        long oldWatermark = Utils.bytes2Long(watermarkBytes);\n\n        if (watermark > oldWatermark) {\n            this.commitWatermark.apply(watermark, stateTopicMessageQueue);\n            Set<MessageQueue> set = new HashSet<>();\n            set.add(stateTopicMessageQueue);\n            stateStore.persist(set);\n        }\n    }\n}\n"
  },
  {
    "path": "core/src/main/java/org/apache/rocketmq/streams/core/window/fire/WindowFire.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.window.fire;\n\nimport org.apache.rocketmq.streams.core.metadata.Data;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\n\nimport java.util.List;\n\npublic interface WindowFire<K, V> {\n\n    List<WindowKey> fire(String operatorName, long watermark);\n\n\n    @SuppressWarnings(\"unchecked\")\n    default Data<K, V> convert(Data<?, ?> data) {\n        return (Data<K, V>) new Data<>(data.getKey(), data.getValue(), data.getTimestamp(), data.getHeader());\n    }\n}\n"
  },
  {
    "path": "core/src/main/resources/logback.xml",
    "content": "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n<configuration>\n    <appender name=\"DefaultAppender_inner\"\n              class=\"ch.qos.logback.core.rolling.RollingFileAppender\">\n        <file>${user.home}/logs/rocketmq-streams/streams_default.log</file>\n        <append>true</append>\n        <rollingPolicy class=\"ch.qos.logback.core.rolling.FixedWindowRollingPolicy\">\n            <fileNamePattern>${user.home}/logs/rocketmq-streams/otherdays/streams_default.%i.log\n            </fileNamePattern>\n            <minIndex>1</minIndex>\n            <maxIndex>10</maxIndex>\n        </rollingPolicy>\n        <triggeringPolicy\n                class=\"ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy\">\n            <maxFileSize>100MB</maxFileSize>\n        </triggeringPolicy>\n        <encoder>\n            <pattern>%d{yyy-MM-dd HH:mm:ss,GMT+8} %p - %m%n</pattern>\n            <charset class=\"java.nio.charset.Charset\">UTF-8</charset>\n        </encoder>\n    </appender>\n    <appender name=\"DefaultAppender\" class=\"ch.qos.logback.classic.AsyncAppender\">\n        <appender-ref ref=\"DefaultAppender_inner\"/>\n    </appender>\n\n    <appender name=\"console\" class=\"ch.qos.logback.core.ConsoleAppender\">\n        <!--设置输出格式-->\n        <encoder class=\"ch.qos.logback.classic.encoder.PatternLayoutEncoder\">\n            <!--格式化输出：%d表示日期，%thread表示线程名，%-5level：级别从左显示5个字符宽度%msg：日志消息，%n是换行符-->\n            <pattern>%d{yyyy-MM-dd HH:mm:ss.SSS} [%thread] %-5level %logger{50} - %msg%n</pattern>\n            <!--设置编码-->\n            <charset>UTF-8</charset>\n        </encoder>\n    </appender>\n\n    <root>\n        <level value=\"INFO\"/>\n        <appender-ref ref=\"console\"/>\n        <appender-ref ref=\"DefaultAppender\"/>\n    </root>\n\n</configuration>\n"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/Num.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core;\n\npublic class Num {\n    private Integer number;\n\n    public Integer getNumber() {\n        return number;\n    }\n\n    public void setNumber(Integer number) {\n        this.number = number;\n    }\n}\n"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/Test.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core;\n\npublic class Test {\n}\n"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/User.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core;\n\npublic class User {\n    private String name;\n\n    public String getName() {\n        return name;\n    }\n\n    public void setName(String name) {\n        this.name = name;\n    }\n}\n"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/function/AggregateSupplier.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.function;\n\nimport com.fasterxml.jackson.core.JsonProcessingException;\nimport io.netty.buffer.ByteBuf;\nimport io.netty.buffer.Unpooled;\nimport org.apache.rocketmq.streams.core.window.WindowState;\nimport org.apache.rocketmq.streams.core.Num;\nimport org.apache.rocketmq.streams.core.User;\nimport org.apache.rocketmq.streams.core.util.Utils;\n\nimport java.nio.charset.StandardCharsets;\n\npublic class AggregateSupplier {\n    public static void main(String[] args) throws Throwable {\n        WindowState<Num, User> state = new WindowState<>();\n        Num num = new Num();\n        num.setNumber(10);\n        User user = new User();\n        user.setName(\"zeni\");\n        state.setKey(num);\n        state.setValue(user);\n\n        byte[] bytes = object2Byte(state);\n\n        WindowState<Num, User> result = byte2Object(bytes);\n\n        System.out.println(result);\n    }\n    private static final ByteBuf buf = Unpooled.buffer(16);\n    protected static byte[] object2Byte(Object obj) throws JsonProcessingException {\n        if (obj == null) {\n            return new byte[]{};\n        }\n\n        String name = obj.getClass().getName();\n        byte[] className = name.getBytes(StandardCharsets.UTF_8);\n        byte[] objBytes = Utils.object2Byte(obj);\n\n\n        buf.writeInt(className.length);\n        buf.writeBytes(className);\n        buf.writeInt(objBytes.length);\n        buf.writeBytes(objBytes);\n\n\n        byte[] bytes = new byte[buf.readableBytes()];\n        buf.readBytes(bytes);\n\n        buf.clear();\n        buf.release();\n        return bytes;\n    }\n\n    /**\n     * decode\n     * <pre>\n     * +-----------+---------------+-------------+-------------+\n     * | Int(4)    |   className  | Int(4)       | value bytes |\n     * | classname |              |object length |             |\n     * +-----------+--------------+---------------+-------------+\n     * </pre>\n     */\n    @SuppressWarnings(\"unchecked\")\n    public static  <V> V byte2Object(byte[] bytes) throws Throwable {\n        ByteBuf byteBuf = Unpooled.wrappedBuffer(bytes);\n\n        int classNameLength = byteBuf.readInt();\n        ByteBuf classNameBuf = byteBuf.readBytes(classNameLength);\n\n        byte[] clazzNameBytes = new byte[classNameBuf.readableBytes()];\n        classNameBuf.readBytes(clazzNameBytes);\n        //实例化\n        String className = new String(clazzNameBytes, StandardCharsets.UTF_8);\n        Class<V> clazz = (Class<V>)Class.forName(className);\n\n        int objectLength = byteBuf.readInt();\n        ByteBuf objBuf = byteBuf.readBytes(objectLength);\n        byte[] objectBytes = new byte[objectLength];\n        objBuf.readBytes(objectBytes);\n\n        return Utils.byte2Object(objectBytes, clazz);\n    }\n\n}\n"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/runtime/WindowStateTests.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.runtime;\n\nimport org.apache.rocketmq.streams.core.window.WindowState;\nimport org.apache.rocketmq.streams.core.Num;\nimport org.apache.rocketmq.streams.core.User;\n\npublic class WindowStateTests {\n    public static void main(String[] args) throws Throwable {\n        WindowState<Num, User> state = new WindowState<>();\n        Num num = new Num();\n        num.setNumber(10);\n        User user = new User();\n        user.setName(\"zeni\");\n        state.setKey(num);\n        state.setValue(user);\n\n        byte[] bytes = WindowState.windowState2Byte(state);\n\n        WindowState<Num, User> state1 = WindowState.byte2WindowState(bytes);\n\n        System.out.println(state1);\n    }\n\n}\n"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/state/RocksDBStoreTest.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.state;\n\nimport org.apache.rocketmq.streams.core.util.ColumnFamilyUtil;\nimport org.apache.rocketmq.streams.core.window.WindowKey;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.util.Utils;\n\nimport java.nio.charset.StandardCharsets;\nimport java.util.List;\n\npublic class RocksDBStoreTest {\n    public static void main(String[] args) throws Throwable {\n        RocksDBStore rocksDBStore = new RocksDBStore(\"test\");\n\n//        String key = \"time@1668249210000@1668249195000\";\n//        String key2 = \"ewwwwe@1668249600481@1\";\n        WindowKey key1 = new WindowKey(\"test1\", \"keyString1\", 10l, 1l);\n        WindowKey key2 = new WindowKey(\"test1\", \"keyString2\", 20l, 1l);\n        Object value1 = \"3\";\n        Object value2 = \"2\";\n\n        byte[] keyBytes = key2Byte(key1);\n        byte[] valueBytes = Utils.object2Byte(value1);\n\n        byte[] keyBytes2 = key2Byte(key2);\n        byte[] valueBytes2 = Utils.object2Byte(value2);\n\n        rocksDBStore.put(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes2), keyBytes2, valueBytes2);\n        rocksDBStore.put(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes), keyBytes, valueBytes);\n\n\n        byte[] bytes = rocksDBStore.get(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes), keyBytes);\n        Object result = Utils.byte2Object(bytes, Object.class);\n        System.out.println(result);\n\n        byte[] bytes2 = rocksDBStore.get(ColumnFamilyUtil.getColumnFamilyByKey(keyBytes2), keyBytes2);\n        Object result2 = Utils.byte2Object(bytes2, Object.class);\n        System.out.println(result2);\n\n        WindowKey searchKey = new WindowKey(\"test1\", \"keyString1\", 13l, 1l);\n        String operatorName = searchKey.getOperatorName();\n        List<Pair<byte[], byte[]>> pairs = rocksDBStore.searchStateLessThanWatermark(operatorName, 11l, RocksDBStoreTest::byte2WindowKey);\n\n        System.out.println(pairs.size());\n    }\n\n    private static WindowKey byte2WindowKey(byte[] source) {\n        String str = new String(source, StandardCharsets.UTF_8);\n        String[] split = Utils.split(str);\n        return new WindowKey(split[0], split[1], Long.parseLong(split[2]), Long.parseLong(split[3]));\n    }\n\n    private static byte[] key2Byte(WindowKey windowKey) {\n        if (windowKey == null) {\n            return new byte[0];\n        }\n\n        return windowKey.toString().getBytes(StandardCharsets.UTF_8);\n    }\n}\n"
  },
  {
    "path": "core/src/test/java/org/apache/rocketmq/streams/core/util/UtilsTests.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.core.util;\n\nimport com.fasterxml.jackson.core.JsonProcessingException;\nimport com.fasterxml.jackson.databind.ObjectMapper;\n\nimport java.nio.charset.StandardCharsets;\n\npublic class UtilsTests {\n    private static final ObjectMapper objectMapper = new ObjectMapper();\n    public static void main(String[] args) throws JsonProcessingException {\n        byte[] bytes = \"tests\".getBytes(StandardCharsets.UTF_8);\n        String tests = \"tests\";\n        byte[] bytes1 = objectMapper.writeValueAsBytes(new User());\n        String s = Utils.toHexString(bytes1);\n        System.out.println(s);\n\n        String s1 = new String(bytes1, StandardCharsets.UTF_8);\n        System.out.println(s1);\n    }\n\n    static class User {\n        private String name;\n\n        public String getName() {\n            return name;\n        }\n\n        public void setName(String name) {\n            this.name = name;\n        }\n    }\n}\n"
  },
  {
    "path": "example/pom.xml",
    "content": "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n<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\">\n    <parent>\n        <artifactId>rocketmq-streams-all</artifactId>\n        <groupId>org.apache.rocketmq</groupId>\n        <version>1.1.2-SNAPSHOT</version>\n    </parent>\n    <modelVersion>4.0.0</modelVersion>\n\n    <artifactId>rocketmq-streams-example</artifactId>\n\n    <properties>\n        <maven.compiler.source>8</maven.compiler.source>\n        <maven.compiler.target>8</maven.compiler.target>\n    </properties>\n    <dependencies>\n        <dependency>\n            <groupId>org.apache.rocketmq</groupId>\n            <artifactId>rocketmq-streams</artifactId>\n            <version>${version}</version>\n        </dependency>\n\n    </dependencies>\n</project>"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/WordCount.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples;\n\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\n\nimport java.nio.charset.StandardCharsets;\nimport java.util.Arrays;\nimport java.util.List;\nimport java.util.Properties;\n\npublic class WordCount {\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"wordCount\");\n\n        builder.source(\"sourceTopic\", total -> {\n                    String value = new String(total, StandardCharsets.UTF_8);\n                    return new Pair<>(null, value);\n                })\n                .flatMap((ValueMapperAction<String, List<String>>) value -> {\n                    String[] splits = value.toLowerCase().split(\",\");\n                    return Arrays.asList(splits);\n                })\n                .keyBy(value -> value)\n                .count()\n                .toRStream()\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.put(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n\n\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/WordCountFromFirstOffset.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples;\n\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.metadata.StreamConfig;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\n\nimport java.nio.charset.StandardCharsets;\nimport java.util.Arrays;\nimport java.util.List;\nimport java.util.Properties;\n\nimport static org.apache.rocketmq.common.consumer.ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET;\n\npublic class WordCountFromFirstOffset {\n    public static void main(String[] args) {\n        StreamBuilder builder = new StreamBuilder(\"wordCount\");\n\n        builder.source(\"sourceTopic\", total -> {\n            String value = new String(total, StandardCharsets.UTF_8);\n            return new Pair<>(null, value);\n        })\n                .flatMap((ValueMapperAction<String, List<String>>) value -> {\n                    String[] splits = value.toLowerCase().split(\"\\\\W+\");\n                    return Arrays.asList(splits);\n                })\n                .keyBy(value -> value)\n                .count()\n                .toRStream()\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.put(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n        properties.put(StreamConfig.ROCKETMQ_STREAMS_CONSUMER_FORM_WHERE, CONSUME_FROM_FIRST_OFFSET);\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n\n\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/joinWindow/JoinWindow.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.joinWindow;\n\nimport com.alibaba.fastjson.JSON;\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.function.ValueJoinAction;\nimport org.apache.rocketmq.streams.core.rstream.RStream;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.window.Time;\nimport org.apache.rocketmq.streams.core.window.WindowBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.examples.pojo.Num;\nimport org.apache.rocketmq.streams.examples.pojo.Union;\nimport org.apache.rocketmq.streams.examples.pojo.User;\n\nimport java.util.Properties;\n\n/**\n * 1、启动RocketMQ\n * 2、创建topic\n * 3、启动本例子运行\n * 4、向topic中写入数据\n * 5、观察输出结果\n */\npublic class JoinWindow {\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"joinWindow\");\n\n        RStream<User> user = builder.source(\"user\", total -> {\n            User user1 = JSON.parseObject(total, User.class);\n            return new Pair<>(null, user1);\n        });\n\n        RStream<Num> num = builder.source(\"num\", source -> {\n            Num user12 = JSON.parseObject(source, Num.class);\n            return new Pair<>(null, user12);\n        });\n\n        ValueJoinAction<User, Num, Union> action = new ValueJoinAction<User, Num, Union>() {\n            @Override\n            public Union apply(User value1, Num value2) {\n                if (value1 != null && value2 != null) {\n                    System.out.println(\"name in user: \" + value1.getName());\n                    System.out.println(\"name in num: \" + value2.getName());\n\n                    return new Union(value1.getName(), value1.getAge(), value2.getNum());\n                }\n\n                if (value2 != null) {\n                    System.out.println(\"name in num: \" + value2.getName());\n                    return new Union(value2.getName(), 0, value2.getNum());\n                }\n\n\n                if (value1 != null) {\n                    System.out.println(\"name in num: \" + value1.getName());\n                    return new Union(value1.getName(), value1.getAge(), 0);\n                }\n\n                throw new IllegalStateException();\n            }\n        };\n\n        user.join(num)\n                .where(User::getName)\n                .equalTo(Num::getName)\n                .window(WindowBuilder.tumblingWindow(Time.seconds(10)))\n                .apply(action)\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.put(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/joinWindow/JoinWithoutWindow.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.joinWindow;\n\nimport com.alibaba.fastjson.JSON;\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.function.ValueJoinAction;\nimport org.apache.rocketmq.streams.core.rstream.RStream;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.examples.pojo.Num;\nimport org.apache.rocketmq.streams.examples.pojo.Union;\nimport org.apache.rocketmq.streams.examples.pojo.User;\n\nimport java.util.Properties;\n\npublic class JoinWithoutWindow {\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"JoinWithoutWindow\");\n\n        RStream<User> user = builder.source(\"user\", total -> {\n            User user1 = JSON.parseObject(total, User.class);\n            return new Pair<>(null, user1);\n        });\n\n        RStream<Num> num = builder.source(\"num\", source -> {\n            Num user12 = JSON.parseObject(source, Num.class);\n            return new Pair<>(null, user12);\n        });\n\n        ValueJoinAction<User, Num, Union> action = new ValueJoinAction<User, Num, Union>() {\n            @Override\n            public Union apply(User value1, Num value2) {\n                if (value1 != null && value2 != null) {\n                    System.out.println(\"name in user: \" + value1.getName());\n                    System.out.println(\"name in num: \" + value2.getName());\n\n                    return new Union(value1.getName(), value1.getAge(), value2.getNum());\n                }\n\n                if (value2 != null) {\n                    System.out.println(\"name in num: \" + value2.getName());\n                    return new Union(value2.getName(), 0, value2.getNum());\n                }\n\n\n                if (value1 != null) {\n                    System.out.println(\"name in num: \" + value1.getName());\n                    return new Union(value1.getName(), value1.getAge(), 0);\n                }\n\n                throw new IllegalStateException();\n            }\n        };\n\n        user.join(num)\n                .where(User::getName)\n                .equalTo(Num::getName)\n                .apply(action)\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.put(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Demo.java",
    "content": "package org.apache.rocketmq.streams.examples.pojo;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n\nimport com.alibaba.fastjson.JSON;\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueDeserializer;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\n\nimport java.util.Properties;\n\n/**\n * 1、启动RocketMQ\n * 2、创建topic\n * 3、启动本例子运行\n * 4、向topic中写入数据\n * 5、观察输出结果\n */\npublic class Demo {\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"demo\");\n\n        builder.source(\"user\", new KeyValueDeserializer<Void, User>() {\n                    @Override\n                    public Pair<Void, User> deserialize(byte[] total) throws Throwable {\n                        //对象需要有默认构造器\n                        User user = JSON.parseObject(total, User.class);\n                        return new Pair<>(null, user);\n                    }\n                })\n                .keyBy(User::getAge)\n                .count(User::getName)\n                .toRStream()\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.put(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Grade.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.pojo;\n\npublic class Grade {\n    private int grade;\n\n    private int score;\n\n    private long timestamp;\n\n    public Grade(int grade, int score) {\n        this.grade = grade;\n        this.score = score;\n    }\n\n    public Grade() {\n\n    }\n\n    public int getGrade() {\n        return grade;\n    }\n\n    public void setGrade(int grade) {\n        this.grade = grade;\n    }\n\n    public int getScore() {\n        return score;\n    }\n\n    public void setScore(int score) {\n        this.score = score;\n    }\n\n    public long getTimestamp() {\n        return timestamp;\n    }\n\n    public void setTimestamp(long timestamp) {\n        this.timestamp = timestamp;\n    }\n\n    @Override public String toString() {\n        return \"Grade{\" +\n            \"grade=\" + grade +\n            \", score=\" + score +\n            \", timestamp=\" + timestamp +\n            '}';\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Num.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.pojo;\n\npublic class Num {\n    private String name;\n    private Integer num;\n\n    public Num(){\n    }\n\n    public Num(String name, Integer num) {\n        this.name = name;\n        this.num = num;\n    }\n\n    public String getName() {\n        return name;\n    }\n\n    public void setName(String name) {\n        this.name = name;\n    }\n\n    public Integer getNum() {\n        return num;\n    }\n\n    public void setNum(Integer num) {\n        this.num = num;\n    }\n\n    @Override\n    public String toString() {\n        return \"Num{\" +\n                \"name='\" + name + '\\'' +\n                \", num=\" + num +\n                '}';\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Order.java",
    "content": "package org.apache.rocketmq.streams.examples.pojo;\n\npublic class Order {\n    private String type;        // drink, food, \n    private Integer price;          // order price\n    private String customer;    // customer name\n\n    public Order() {\n\n    }\n\n    public Order(String type, Integer price, String customer) {\n        this.type = type;\n        this.price = price;\n        this.customer = customer;\n    }\n\n    public String getType() {\n        return type;\n    }\n\n    public String getCustomer() {\n        return customer;\n    }\n\n    public Integer getPrice() {\n        return price;\n    }\n\n    public void setType(String type) {\n        this.type = type;\n    }\n\n    public void setCustomer(String customer) {\n        this.customer = customer;\n    }\n\n    public void setPrice(Integer price) {\n        this.price = price;\n    }\n\n    @Override\n    public String toString() {\n        return \"Order{\" + \"type=\" + type + \", price=\" + price + \", customer=\" + customer + \"}\";\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/Union.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.pojo;\n\npublic class Union {\n    private String name;\n    private Integer age;\n    private Integer num;\n\n    public Union(String name, Integer age, Integer num) {\n        this.name = name;\n        this.age = age;\n        this.num = num;\n    }\n\n    public Union() {\n    }\n\n    public String getName() {\n        return name;\n    }\n\n    public void setName(String name) {\n        this.name = name;\n    }\n\n    public Integer getAge() {\n        return age;\n    }\n\n    public void setAge(Integer age) {\n        this.age = age;\n    }\n\n    public Integer getNum() {\n        return num;\n    }\n\n    public void setNum(Integer num) {\n        this.num = num;\n    }\n\n    @Override\n    public String toString() {\n        return \"Union{\" +\n                \"name='\" + name + '\\'' +\n                \", age=\" + age +\n                \", num=\" + num +\n                '}';\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/pojo/User.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.pojo;\n\npublic class User {\n    private String name;\n    private Integer age;\n    private long timestamp;\n\n    //序列化/反序列化使用\n    public User() {\n    }\n\n    public User(String name, Integer age) {\n        this.name = name;\n        this.age = age;\n    }\n\n    public User(String name, Integer age, long timestamp) {\n        this.name = name;\n        this.age = age;\n        this.timestamp = timestamp;\n    }\n\n    public String getName() {\n        return name;\n    }\n\n    public void setName(String name) {\n        this.name = name;\n    }\n\n    public Integer getAge() {\n        return age;\n    }\n\n    public void setAge(Integer age) {\n        this.age = age;\n    }\n\n    public long getTimestamp() {\n        return timestamp;\n    }\n\n    public void setTimestamp(long timestamp) {\n        this.timestamp = timestamp;\n    }\n\n    @Override\n    public String toString() {\n        return \"User{\" +\n                \"name='\" + name + '\\'' +\n                \", age=\" + age +\n                \", timeStamp=\" + timestamp +\n                '}';\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/sink/WordCountSink.java",
    "content": "package org.apache.rocketmq.streams.examples.sink;\n/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nimport com.fasterxml.jackson.databind.ObjectMapper;\nimport com.fasterxml.jackson.databind.node.ObjectNode;\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.serialization.KeyValueSerializer;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\n\nimport java.nio.charset.StandardCharsets;\nimport java.util.Arrays;\nimport java.util.List;\nimport java.util.Properties;\n\n/**\n * 1、启动RocketMQ\n * 2、创建topic sh bin/mqadmin updateTopic -c DefaultCluster -t sourceTopic -r 8 -w 8 -n 127.0.0.1:9876\n * 3、启动本例子运行\n * 4、向topic中写入数据\n * 5、观察输出结果\n */\npublic class WordCountSink {\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"wordCount\");\n\n        builder.source(\"sourceTopic\", total -> {\n                    String value = new String(total, StandardCharsets.UTF_8);\n                    return new Pair<>(null, value);\n                })\n                .flatMap((ValueMapperAction<String, List<String>>) value -> {\n                    String[] splits = value.toLowerCase().split(\"\\\\W+\");\n                    return Arrays.asList(splits);\n                })\n                .keyBy(value -> value)\n                .count()\n                .sink(\"wordCountSink\", new KeyValueSerializer<String, Integer>() {\n                    final ObjectMapper objectMapper = new ObjectMapper();\n\n                    @Override\n                    public byte[] serialize(String o, Integer data) throws Throwable {\n                        ObjectNode objectNode = objectMapper.createObjectNode();\n                        objectNode.put(o, data);\n\n                        String result = objectNode.toPrettyString();\n                        return objectMapper.writeValueAsBytes(result);\n                    }\n                });\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.put(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/SessionWindowCount.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.window;\n\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.window.Time;\nimport org.apache.rocketmq.streams.core.window.WindowBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\n\nimport java.nio.charset.StandardCharsets;\nimport java.text.SimpleDateFormat;\nimport java.util.Date;\nimport java.util.Properties;\n\n/**\n * 1、启动RocketMQ\n * 2、创建topic\n * 3、启动本例子运行\n * 4、向topic中写入数据\n * 5、观察输出结果\n */\npublic class SessionWindowCount {\n    private static final SimpleDateFormat format = new SimpleDateFormat(\"yyyy-MM-dd HH:mm:ss\");\n\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"sessionWindowCount\");\n        builder.source(\"windowCount\", source -> {\n                    String value = new String(source, StandardCharsets.UTF_8);\n                    int result = Integer.parseInt(value);\n\n                    System.out.println(\"time=\" + format.format(new Date(System.currentTimeMillis())) + \", value=\" + value);\n                    return new Pair<>(null, result);\n                })\n                .filter(value -> value > 0)\n                .keyBy(value -> \"key\")\n                .window(WindowBuilder.sessionWindow(Time.seconds(5)))\n                .count()\n                .toRStream()\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/SlideWindowCount.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.window;\n\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.window.Time;\nimport org.apache.rocketmq.streams.core.window.WindowBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\n\nimport java.nio.charset.StandardCharsets;\nimport java.util.Properties;\n\n/**\n * 1、启动RocketMQ\n * 2、创建topic\n * 3、启动本例子运行\n * 4、向topic中写入数据\n * 5、观察输出结果\n */\npublic class SlideWindowCount {\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"slideWindowCount\");\n        builder.source(\"windowCount\", source -> {\n                    String value = new String(source, StandardCharsets.UTF_8);\n                    int result = Integer.parseInt(value);\n                    return new Pair<>(null, result);\n                })\n                .filter(value -> value > 0)\n                .keyBy(value -> \"key\")\n                .window(WindowBuilder.slidingWindow(Time.seconds(5), Time.seconds(2)))\n                .count()\n                .toRStream()\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowAvg.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.window;\n\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.window.Time;\nimport org.apache.rocketmq.streams.core.window.WindowBuilder;\n\nimport java.nio.charset.StandardCharsets;\nimport java.time.LocalTime;\nimport java.util.Properties;\nimport java.util.concurrent.CountDownLatch;\n\n/**\n * 1、启动RocketMQ\n * 2、创建topic\n * 3、启动本例子运行\n * 4、向topic中写入数据\n * 5、观察输出结果\n */\npublic class WindowAvg {\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"WindowAvg\");\n        builder.source(\"avgSource\", source -> {\n            String value = new String(source, StandardCharsets.UTF_8);\n            Integer num = Integer.parseInt(value);\n            return new Pair<>(null, num);\n        }).foreach(value -> System.out.println(String.format(\"time:%s, input:%d\", LocalTime.now(), value)))\n        .filter(value -> value > 0)\n        .keyBy(value -> \"key\")\n        .window(WindowBuilder.tumblingWindow(Time.seconds(5)))\n        .avg()\n        .toRStream()\n        .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n        Runtime.getRuntime().addShutdownHook(new Thread(\"WindowAvg-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowCount.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.window;\n\n\nimport com.alibaba.fastjson.JSON;\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.metadata.StreamConfig;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.window.Time;\nimport org.apache.rocketmq.streams.core.window.TimeType;\nimport org.apache.rocketmq.streams.core.window.WindowBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.examples.pojo.User;\n\nimport java.util.Properties;\n\n/**\n * 1、启动RocketMQ\n * 2、创建topic\n * 3、启动本例子运行\n * 4、向topic中写入数据\n * 5、观察输出结果\n */\npublic class WindowCount {\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"windowCountUser\");\n\n        builder.source(\"user\", source -> {\n                    User user = JSON.parseObject(source, User.class);\n                    return new Pair<>(null, user);\n                })\n                .selectTimestamp(User::getTimestamp)\n                .keyBy(User::getAge)\n                .window(WindowBuilder.tumblingWindow(Time.seconds(5)))\n                .count()\n                .toRStream()\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n        properties.put(StreamConfig.TIME_TYPE, TimeType.EVENT_TIME);\n        properties.put(StreamConfig.ALLOW_LATENESS_MILLISECOND, 2000);\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowMin.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.window;\n\nimport com.alibaba.fastjson.JSON;\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.metadata.StreamConfig;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.window.Time;\nimport org.apache.rocketmq.streams.core.window.WindowBuilder;\nimport org.apache.rocketmq.streams.examples.pojo.Grade;\n\nimport java.text.SimpleDateFormat;\nimport java.util.Date;\nimport java.util.Properties;\n\n/**\n * 1、启动RocketMQ\n * 2、创建topic\n * 3、启动本例子运行\n * 4、向topic中写入数据\n * 5、观察输出结果\n */\npublic class WindowMin {\n    private static final SimpleDateFormat format = new SimpleDateFormat(\"yyyy-MM-dd HH:mm:ss\");\n\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"windowMinGrade\");\n\n        builder.source(\"grade\", source -> {\n                    Grade grade = JSON.parseObject(source, Grade.class);\n                    System.out.println(\"time=\" + format.format(new Date(System.currentTimeMillis())) + \",\" + grade);\n                    return new Pair<>(null, grade);\n                })\n                .keyBy(Grade::getGrade)\n                .window(WindowBuilder.tumblingWindow(Time.seconds(5)))\n                .min(Grade::getScore) //.max(Grade::getScore)\n                .toRStream()\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n        properties.put(StreamConfig.ALLOW_LATENESS_MILLISECOND, 2000);\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n}\n\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowOrderCount.java",
    "content": "package org.apache.rocketmq.streams.examples.window;\n\nimport com.alibaba.fastjson.JSON;\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.function.SelectAction;\nimport org.apache.rocketmq.streams.core.metadata.StreamConfig;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.window.Time;\nimport org.apache.rocketmq.streams.core.window.WindowBuilder;\nimport org.apache.rocketmq.streams.examples.pojo.Order;\n\nimport java.util.Properties;\n\npublic class WindowOrderCount {\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = getOrder2();\n\n        TopologyBuilder topologyBuilder = builder.build();\n        Properties properties = new Properties();\n        properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n        properties.put(StreamConfig.ALLOW_LATENESS_MILLISECOND, 2000);\n\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n\n        Runtime.getRuntime().addShutdownHook(new Thread(\"ordercount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n\n    private static StreamBuilder getOrder1() throws Throwable {\n        /**\n         * Get the count of drink/food orders in last 30 seconds every 10 seconds\n         **/\n        StreamBuilder builder = new StreamBuilder(\"windowOrderCount\");\n\n        builder.source(\"order\", source -> {\n                    Order order = JSON.parseObject(source, Order.class);\n                    System.out.println(order.toString());\n                    return new Pair<>(null, order);\n                })\n                .keyBy(Order::getType)\n                .window(WindowBuilder.slidingWindow(Time.seconds(30), Time.seconds(10)))\n                .count()\n                .toRStream()\n                .print();\n\n        return builder;\n    }\n\n    private static StreamBuilder getOrder2() throws Throwable {\n        /**\n         * Get how much the customers pay for drink/food every 100 seconds\n         **/\n        StreamBuilder builder = new StreamBuilder(\"windowOrderCount\");\n        builder.source(\"order\", source -> {\n                    Order order = JSON.parseObject(source, Order.class);\n                    System.out.println(order.toString());\n                    return new Pair<>(null, order);\n                })\n                .keyBy(new SelectAction<String, Order>() {\n                    @Override\n                    public String select(Order order) {\n                        return order.getCustomer() + \"@\" + order.getType();\n                    }\n                })\n                .window(WindowBuilder.tumblingWindow(Time.seconds(100)))\n                .sum(Order::getPrice)\n                .toRStream()\n                .print();\n\n        return builder;\n    }\n}\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowSum.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.window;\n\nimport com.alibaba.fastjson.JSON;\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.metadata.StreamConfig;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\nimport org.apache.rocketmq.streams.core.window.Time;\nimport org.apache.rocketmq.streams.core.window.WindowBuilder;\nimport org.apache.rocketmq.streams.examples.pojo.Grade;\n\nimport java.text.SimpleDateFormat;\nimport java.util.Date;\nimport java.util.Properties;\n\n/**\n * 1、启动RocketMQ\n * 2、创建topic\n * 3、启动本例子运行\n * 4、向topic中写入数据\n * 5、观察输出结果\n */\npublic class WindowSum {\n    private static final SimpleDateFormat format = new SimpleDateFormat(\"yyyy-MM-dd HH:mm:ss\");\n\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"windowSumGrade\");\n\n        builder.source(\"grade\", source -> {\n                    Grade grade = JSON.parseObject(source, Grade.class);\n                    System.out.println(\"time=\" + format.format(new Date(System.currentTimeMillis())) + \",\" + grade);\n                    return new Pair<>(null, grade);\n                })\n                .keyBy(Grade::getGrade)\n                .window(WindowBuilder.tumblingWindow(Time.seconds(5)))\n                .sum(Grade::getScore)\n                .toRStream()\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n        properties.put(StreamConfig.ALLOW_LATENESS_MILLISECOND, 2000);\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n}\n\n"
  },
  {
    "path": "example/src/main/java/org/apache/rocketmq/streams/examples/window/WindowWordCount.java",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *     http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\npackage org.apache.rocketmq.streams.examples.window;\n\nimport org.apache.rocketmq.common.MixAll;\nimport org.apache.rocketmq.streams.core.RocketMQStream;\nimport org.apache.rocketmq.streams.core.function.ValueMapperAction;\nimport org.apache.rocketmq.streams.core.metadata.StreamConfig;\nimport org.apache.rocketmq.streams.core.rstream.StreamBuilder;\nimport org.apache.rocketmq.streams.core.window.Time;\nimport org.apache.rocketmq.streams.core.window.TimeType;\nimport org.apache.rocketmq.streams.core.window.WindowBuilder;\nimport org.apache.rocketmq.streams.core.topology.TopologyBuilder;\nimport org.apache.rocketmq.streams.core.util.Pair;\n\nimport java.nio.charset.StandardCharsets;\nimport java.util.Arrays;\nimport java.util.List;\nimport java.util.Properties;\n\n/**\n * 1、启动RocketMQ\n * 2、创建topic\n * 3、启动本例子运行\n * 4、向topic中写入数据\n * 5、观察输出结果\n */\npublic class WindowWordCount {\n    public static void main(String[] args) throws Throwable {\n        StreamBuilder builder = new StreamBuilder(\"windowWordCount\");\n        builder.source(\"sourceTopic\", source -> {\n                    String value = new String(source, StandardCharsets.UTF_8);\n                    return new Pair<>(null, value);\n                })\n                .flatMap((ValueMapperAction<String, List<String>>) value -> {\n                    String[] splits = value.toLowerCase().split(\"\\\\W+\");\n                    return Arrays.asList(splits);\n                })\n                .keyBy(value -> value)\n                .window(WindowBuilder.tumblingWindow(Time.seconds(5)))\n                .count()\n                .toRStream()\n                .print();\n\n        TopologyBuilder topologyBuilder = builder.build();\n\n        Properties properties = new Properties();\n        properties.putIfAbsent(MixAll.NAMESRV_ADDR_PROPERTY, \"127.0.0.1:9876\");\n        properties.put(StreamConfig.TIME_TYPE, TimeType.PROCESS_TIME);\n\n        RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);\n\n        Runtime.getRuntime().addShutdownHook(new Thread(\"wordcount-shutdown-hook\") {\n            @Override\n            public void run() {\n                rocketMQStream.stop();\n            }\n        });\n\n        rocketMQStream.start();\n    }\n}\n"
  },
  {
    "path": "pom.xml",
    "content": "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n<!--\n  Licensed to the Apache Software Foundation (ASF) under one or more\n  contributor license agreements.  See the NOTICE file distributed with\n  this work for additional information regarding copyright ownership.\n  The ASF licenses this file to You under the Apache License, Version 2.0\n  (the \"License\"); you may not use this file except in compliance with\n  the License.  You may obtain a copy of the License at\n\n      http://www.apache.org/licenses/LICENSE-2.0\n\n  Unless required by applicable law or agreed to in writing, software\n  distributed under the License is distributed on an \"AS IS\" BASIS,\n  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n  See the License for the specific language governing permissions and\n  limitations under the License.\n  -->\n<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\">\n    <modelVersion>4.0.0</modelVersion>\n    <parent>\n        <groupId>org.apache</groupId>\n        <artifactId>apache</artifactId>\n        <version>18</version>\n    </parent>\n\n\n    <groupId>org.apache.rocketmq</groupId>\n    <artifactId>rocketmq-streams-all</artifactId>\n    <version>1.1.2-SNAPSHOT</version>\n    <name>Apache RocketMQ Streams ${project.version}</name>\n    <packaging>pom</packaging>\n    <url>https://rocketmq.apache.org/</url>\n\n    <scm>\n        <url>git@github.com:apache/rocketmq-streams.git</url>\n        <connection>scm:git:git@github.com:apache/rocketmq-streams.git</connection>\n        <developerConnection>scm:git:git@github.com:apache/rocketmq-streams.git</developerConnection>\n        <tag>HEAD</tag>\n    </scm>\n\n    <mailingLists>\n        <mailingList>\n            <name>Development List</name>\n            <subscribe>dev-subscribe@rocketmq.apache.org</subscribe>\n            <unsubscribe>dev-unsubscribe@rocketmq.apache.org</unsubscribe>\n            <post>dev@rocketmq.apache.org</post>\n        </mailingList>\n        <mailingList>\n            <name>User List</name>\n            <subscribe>users-subscribe@rocketmq.apache.org</subscribe>\n            <unsubscribe>users-unsubscribe@rocketmq.apache.org</unsubscribe>\n            <post>users@rocketmq.apache.org</post>\n        </mailingList>\n        <mailingList>\n            <name>Commits List</name>\n            <subscribe>commits-subscribe@rocketmq.apache.org</subscribe>\n            <unsubscribe>commits-unsubscribe@rocketmq.apache.org</unsubscribe>\n            <post>commits@rocketmq.apache.org</post>\n        </mailingList>\n    </mailingLists>\n\n    <developers>\n        <developer>\n            <id>Apache RocketMQ</id>\n            <name>Apache RocketMQ of ASF</name>\n            <url>https://rocketmq.apache.org/</url>\n        </developer>\n    </developers>\n\n    <organization>\n        <name>Apache Software Foundation</name>\n        <url>http://www.apache.org</url>\n    </organization>\n\n    <licenses>\n        <license>\n            <name>Apache License, Version 2.0</name>\n            <url>http://www.apache.org/licenses/LICENSE-2.0</url>\n            <distribution>repo</distribution>\n        </license>\n    </licenses>\n\n    <modules>\n        <module>core</module>\n        <module>example</module>\n    </modules>\n\n    <properties>\n        <rocketmq.version>5.0.0</rocketmq.version>\n        <rocksdbjni.version>7.6.0</rocksdbjni.version>\n        <maven.compiler.source>8</maven.compiler.source>\n        <maven.compiler.target>8</maven.compiler.target>\n    </properties>\n\n\n    <dependencyManagement>\n        <dependencies>\n            <!-- ================================================= -->\n            <!-- rocketmq library -->\n            <!-- ================================================= -->\n\n            <dependency>\n                <groupId>org.apache.rocketmq</groupId>\n                <artifactId>rocketmq-tools</artifactId>\n                <version>${rocketmq.version}</version>\n            </dependency>\n            <dependency>\n                <groupId>org.apache.rocketmq</groupId>\n                <artifactId>rocketmq-common</artifactId>\n                <version>${rocketmq.version}</version>\n            </dependency>\n            <dependency>\n                <groupId>org.apache.rocketmq</groupId>\n                <artifactId>rocketmq-client</artifactId>\n                <version>${rocketmq.version}</version>\n            </dependency>\n            <dependency>\n                <groupId>org.apache.rocketmq</groupId>\n                <artifactId>rocketmq-logging</artifactId>\n                <version>${rocketmq.version}</version>\n            </dependency>\n\n            <dependency>\n                <groupId>org.rocksdb</groupId>\n                <artifactId>rocksdbjni</artifactId>\n                <version>${rocksdbjni.version}</version>\n            </dependency>\n        </dependencies>\n    </dependencyManagement>\n\n    <build>\n        <plugins>\n            <plugin>\n                <groupId>org.apache.maven.plugins</groupId>\n                <artifactId>maven-surefire-plugin</artifactId>\n                <version>2.18.1</version>\n                <configuration>\n                    <skipTests>true</skipTests>\n                </configuration>\n            </plugin>\n            <plugin>\n                <groupId>org.apache.rat</groupId>\n                <artifactId>apache-rat-plugin</artifactId>\n                <version>0.12</version>\n                <configuration>\n                    <excludes>\n                        <exclude>.gitignore</exclude>\n                        <exclude>.travis.yml</exclude>\n                        <exclude>.asf.yaml</exclude>\n                        <exclude>build_without_test.sh</exclude>\n                        <exclude>NOTICE</exclude>\n                        <exclude>LICENSE</exclude>\n                        <exclude>README.md</exclude>\n                        <exclude>.github/**</exclude>\n                        <exclude>*/target/**</exclude>\n                        <exclude>*/*.iml</exclude>\n                        <exclude>**/*.txt</exclude>\n                        <exclude>**/*.cs</exclude>\n                        <exclude>**/*.xml</exclude>\n                        <exclude>**/*.sh</exclude>\n                        <exclude>**/*.out</exclude>\n                        <exclude>**/*.properties</exclude>\n                    </excludes>\n                </configuration>\n            </plugin>\n            <plugin>\n                <groupId>org.apache.maven.plugins</groupId>\n                <artifactId>maven-compiler-plugin</artifactId>\n                <version>3.5.1</version>\n                <configuration>\n                    <source>${maven.compiler.source}</source>\n                    <target>${maven.compiler.target}</target>\n                    <showDeprecation>true</showDeprecation>\n                    <showWarnings>true</showWarnings>\n                </configuration>\n            </plugin>\n            <plugin>\n                <groupId>org.codehaus.mojo</groupId>\n                <artifactId>versions-maven-plugin</artifactId>\n                <version>2.2</version>\n            </plugin>\n            <plugin>\n                <groupId>org.codehaus.mojo</groupId>\n                <artifactId>cobertura-maven-plugin</artifactId>\n                <version>2.7</version>\n                <configuration>\n                    <formats>\n                        <format>html</format>\n                        <format>xml</format>\n                    </formats>\n                    <check />\n                </configuration>\n            </plugin>\n            <plugin>\n                <groupId>org.jacoco</groupId>\n                <artifactId>jacoco-maven-plugin</artifactId>\n                <version>0.8.7</version>\n                <executions>\n                    <execution>\n                        <goals>\n                            <goal>prepare-agent</goal>\n                        </goals>\n                    </execution>\n                    <execution>\n                        <id>report</id>\n                        <phase>test</phase>\n                        <goals>\n                            <goal>report</goal>\n                        </goals>\n                    </execution>\n                </executions>\n            </plugin>\n            <plugin>\n                <groupId>org.apache.maven.plugins</groupId>\n                <artifactId>maven-source-plugin</artifactId>\n                <version>3.0.1</version>\n                <executions>\n                    <execution>\n                        <id>attach-sources</id>\n                        <goals>\n                            <goal>jar</goal>\n                        </goals>\n                    </execution>\n                </executions>\n            </plugin>\n        </plugins>\n    </build>\n\n\n</project>\n"
  }
]