Repository: apache/rocketmq-streams
Branch: develop
Commit: 22976de3ef0f
Files: 132
Total size: 452.2 KB
Directory structure:
gitextract_86qvi7ak/
├── .asf.yaml
├── .gitignore
├── .travis.yml
├── LICENSE
├── NOTICE
├── README.md
├── build_without_test.sh
├── core/
│ ├── pom.xml
│ └── src/
│ ├── main/
│ │ ├── java/
│ │ │ └── org/
│ │ │ └── apache/
│ │ │ └── rocketmq/
│ │ │ └── streams/
│ │ │ └── core/
│ │ │ ├── RocketMQStream.java
│ │ │ ├── common/
│ │ │ │ └── Constant.java
│ │ │ ├── exception/
│ │ │ │ ├── DataProcessThrowable.java
│ │ │ │ ├── DeserializeThrowable.java
│ │ │ │ ├── RStreamsException.java
│ │ │ │ └── RecoverStateStoreThrowable.java
│ │ │ ├── function/
│ │ │ │ ├── AggregateAction.java
│ │ │ │ ├── FilterAction.java
│ │ │ │ ├── ForeachAction.java
│ │ │ │ ├── SelectAction.java
│ │ │ │ ├── ValueJoinAction.java
│ │ │ │ ├── ValueMapperAction.java
│ │ │ │ ├── accumulator/
│ │ │ │ │ ├── Accumulator.java
│ │ │ │ │ ├── AvgAccumulator.java
│ │ │ │ │ ├── CountAccumulator.java
│ │ │ │ │ └── MinAccumulator.java
│ │ │ │ └── supplier/
│ │ │ │ ├── AccumulatorSupplier.java
│ │ │ │ ├── AddTagSupplier.java
│ │ │ │ ├── AggregateSupplier.java
│ │ │ │ ├── FilterSupplier.java
│ │ │ │ ├── ForeachSupplier.java
│ │ │ │ ├── JoinAggregateSupplier.java
│ │ │ │ ├── JoinWindowAggregateSupplier.java
│ │ │ │ ├── KeySelectSupplier.java
│ │ │ │ ├── MultiValueChangeSupplier.java
│ │ │ │ ├── PrintSupplier.java
│ │ │ │ ├── SinkSupplier.java
│ │ │ │ ├── SourceSupplier.java
│ │ │ │ ├── SumAggregate.java
│ │ │ │ ├── TimestampSelectorSupplier.java
│ │ │ │ ├── ValueChangeSupplier.java
│ │ │ │ ├── WindowAccumulatorSupplier.java
│ │ │ │ └── WindowAggregateSupplier.java
│ │ │ ├── metadata/
│ │ │ │ ├── Data.java
│ │ │ │ └── StreamConfig.java
│ │ │ ├── rstream/
│ │ │ │ ├── GroupedStream.java
│ │ │ │ ├── GroupedStreamImpl.java
│ │ │ │ ├── JoinedStream.java
│ │ │ │ ├── Pipeline.java
│ │ │ │ ├── RStream.java
│ │ │ │ ├── RStreamImpl.java
│ │ │ │ ├── StreamBuilder.java
│ │ │ │ ├── WindowStream.java
│ │ │ │ └── WindowStreamImpl.java
│ │ │ ├── running/
│ │ │ │ ├── AbstractProcessor.java
│ │ │ │ ├── AbstractWindowProcessor.java
│ │ │ │ ├── MessageQueueListenerWrapper.java
│ │ │ │ ├── Processor.java
│ │ │ │ ├── RocketMQClient.java
│ │ │ │ ├── StreamContext.java
│ │ │ │ ├── StreamContextImpl.java
│ │ │ │ └── WorkerThread.java
│ │ │ ├── serialization/
│ │ │ │ ├── KeyValueDeserializer.java
│ │ │ │ ├── KeyValueSerializer.java
│ │ │ │ ├── ShuffleProtocol.java
│ │ │ │ ├── deImpl/
│ │ │ │ │ └── KVJsonDeserializer.java
│ │ │ │ └── serImpl/
│ │ │ │ └── KVJsonSerializer.java
│ │ │ ├── state/
│ │ │ │ ├── AbstractStore.java
│ │ │ │ ├── RocketMQStore.java
│ │ │ │ ├── RocksDBStore.java
│ │ │ │ └── StateStore.java
│ │ │ ├── topology/
│ │ │ │ ├── TopologyBuilder.java
│ │ │ │ ├── real/
│ │ │ │ │ ├── ProcessorFactory.java
│ │ │ │ │ ├── RealProcessorFactory.java
│ │ │ │ │ ├── SinkFactory.java
│ │ │ │ │ └── SourceFactory.java
│ │ │ │ └── virtual/
│ │ │ │ ├── AbstractGraphNode.java
│ │ │ │ ├── GraphNode.java
│ │ │ │ ├── ProcessorNode.java
│ │ │ │ ├── ShuffleProcessorNode.java
│ │ │ │ ├── SinkGraphNode.java
│ │ │ │ └── SourceGraphNode.java
│ │ │ ├── util/
│ │ │ │ ├── ColumnFamilyUtil.java
│ │ │ │ ├── OperatorNameMaker.java
│ │ │ │ ├── Pair.java
│ │ │ │ ├── RocketMQUtil.java
│ │ │ │ └── Utils.java
│ │ │ └── window/
│ │ │ ├── JoinType.java
│ │ │ ├── StreamType.java
│ │ │ ├── Time.java
│ │ │ ├── TimeType.java
│ │ │ ├── Window.java
│ │ │ ├── WindowBuilder.java
│ │ │ ├── WindowInfo.java
│ │ │ ├── WindowKey.java
│ │ │ ├── WindowState.java
│ │ │ ├── WindowStore.java
│ │ │ └── fire/
│ │ │ ├── AbstractWindowFire.java
│ │ │ ├── AccumulatorSessionWindowFire.java
│ │ │ ├── AccumulatorWindowFire.java
│ │ │ ├── AggregateSessionWindowFire.java
│ │ │ ├── AggregateWindowFire.java
│ │ │ ├── IdleWindowScaner.java
│ │ │ ├── JoinWindowFire.java
│ │ │ └── WindowFire.java
│ │ └── resources/
│ │ └── logback.xml
│ └── test/
│ └── java/
│ └── org/
│ └── apache/
│ └── rocketmq/
│ └── streams/
│ └── core/
│ ├── Num.java
│ ├── Test.java
│ ├── User.java
│ ├── function/
│ │ └── AggregateSupplier.java
│ ├── runtime/
│ │ └── WindowStateTests.java
│ ├── state/
│ │ └── RocksDBStoreTest.java
│ └── util/
│ └── UtilsTests.java
├── example/
│ ├── pom.xml
│ └── src/
│ └── main/
│ └── java/
│ └── org/
│ └── apache/
│ └── rocketmq/
│ └── streams/
│ └── examples/
│ ├── WordCount.java
│ ├── WordCountFromFirstOffset.java
│ ├── joinWindow/
│ │ ├── JoinWindow.java
│ │ └── JoinWithoutWindow.java
│ ├── pojo/
│ │ ├── Demo.java
│ │ ├── Grade.java
│ │ ├── Num.java
│ │ ├── Order.java
│ │ ├── Union.java
│ │ └── User.java
│ ├── sink/
│ │ └── WordCountSink.java
│ └── window/
│ ├── SessionWindowCount.java
│ ├── SlideWindowCount.java
│ ├── WindowAvg.java
│ ├── WindowCount.java
│ ├── WindowMin.java
│ ├── WindowOrderCount.java
│ ├── WindowSum.java
│ └── WindowWordCount.java
└── pom.xml
================================================
FILE CONTENTS
================================================
================================================
FILE: .asf.yaml
================================================
github:
features:
# Enable issue management
issues: true
# Enable wiki
wiki: true
protected_branches:
notifications:
commits: commits@rocketmq.apache.org
issues: commits@rocketmq.apache.org
pullrequests: commits@rocketmq.apache.org
jobs: commits@rocketmq.apache.org
discussions: dev@rocketmq.apache.org
================================================
FILE: .gitignore
================================================
target/
.DS_Store
!.mvn/wrapper/maven-wrapper.jar
*.versionsBackup
.gradle/
### STS ###
.apt_generated
.classpath
.factorypath
.project
.settings
.springBeans
### IntelliJ IDEA ###
.idea
*.iws
*.iml
*.ipr
*.zip
### NetBeans ###
nbproject/private/
build/
nbbuild/
dist/
nbdist/
pom.xml.releaseBackup
release.properties
================================================
FILE: .travis.yml
================================================
dist: trusty
language: java
jdk:
- oraclejdk8
- oraclejdk9
- oraclejdk11
- openjdk8
before_install:
- echo 'MAVEN_OPTS="$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal"' >> ~/.mavenrc
- cat ~/.mavenrc
- if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi
- if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "$CUSTOM_JDK"; fi
script:
- travis_retry mvn -B clean apache-rat:check
- travis_retry mvn -B clean install cobertura:cobertura
after_success:
- bash <(curl -s https://codecov.io/bash) || echo 'Codecov failed to upload'
================================================
FILE: LICENSE
================================================
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
================================================
FILE: NOTICE
================================================
Apache RocketMQ
Copyright 2016-2023 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
================================================
FILE: README.md
================================================
# RocketMQ Streams
[](https://app.travis-ci.com/apache/rocketmq-streams)
[](https://app.codecov.io/gh/apache/rocketmq-streams)
[](https://github.com/apache/rocketmq-streams/releases)
[](https://www.apache.org/licenses/LICENSE-2.0.html)
[](http://isitmaintained.com/project/apache/rocketmq-streams "Average time to resolve an issue")
[](http://isitmaintained.com/project/apache/rocketmq-streams "Percentage of issues still open")
[](https://twitter.com/intent/follow?screen_name=ApacheRocketMQ)
**RocketMQ Streams is a lightweight stream processing framework, application gains the stream processing ability by depending on RocketMQ Streams as an SDK.**
It offers a variety of features:
* Function:
* One-to-one transform function, such as: filter, map, foreach
* Aggregate function, such as: sum, min, max, count, aggregate
* Generating function, such as: flatMap
* Group by aggregate and window aggregate
* Join stream
* Custom serialization
----------
## Quick Start
This paragraph guides you running a stream processing with RocketMQ Streams.
### Run RocketMQ 5.0 locally
[RocketMQ quick-start](https://rocketmq.apache.org/docs/quick-start/)
RocketMQ runs on all major operating systems and requires only a Java JDK version 8 or higher to be installed.
To check, run `java -version`:
```shell
$ java -version
java version "1.8.0_121"
```
**1) Download RocketMQ**
```shell
wget https://archive.apache.org/dist/rocketmq/5.0.0/rocketmq-all-5.0.0-bin-release.zip
# Unpack the release
$ unzip rocketmq-all-5.0.0-bin-release.zip
$ cd rocketmq-all-5.0.0-bin-release/bin
```
**2) Start NameServer**
NameServer will be listening at `0.0.0.0:9876`, make sure that the port is not used by others on the local machine, and then do as follows.
```shell
### start Name Server
$ nohup sh mqnamesrv &
### check whether Name Server is successfully started
$ tail -f ~/logs/rocketmqlogs/namesrv.log
The Name Server boot success...
```
**2) Start Broker**
```shell
### start Broker
$ nohup sh bin/mqbroker -n localhost:9876 &
### check whether Broker is successfully started, eg: Broker's IP is 192.168.1.2, Broker's name is broker-a
$ tail -f ~/logs/rocketmqlogs/broker.log
The broker[broker-a, 192.169.1.2:10911] boot success...
```
### Build stream processing application
**1) Build application in IDE**
**2) Add RocketMQ Streams dependency**
```xml
org.apache.rocketmqrocketmq-streams{current.version}
```
**3) Build stream processing application**
* create topic in RocketMQ before start the stream processing.
```shell
sh bin/mqadmin updateTopic -c ${clusterName} -t ${topicName} -r 8 -w 8 -n 127.0.0.1:9876
```
NOTE: the default clusterName is DefaultCluster in this quick-start doc, changes it with your RocketMQ cluster.
* add your stream processing code, The following is an example. more examples are [here](./example/src/main/java/org/apache/rocketmq/streams/examples).
```java
public static void main(String[] args) {
StreamBuilder builder = new StreamBuilder("wordCount");
builder.source("sourceTopic", total -> {
String value = new String(total, StandardCharsets.UTF_8);
return new Pair<>(null, value);
})
.flatMap((ValueMapperAction>) value -> {
String[] splits = value.toLowerCase().split("\\W+");
return Arrays.asList(splits);
})
.keyBy(value -> value)
.count()
.toRStream()
.print();
TopologyBuilder topologyBuilder = builder.build();
Properties properties = new Properties();
properties.put(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:9876");
RocketMQStream rocketMQStream = new RocketMQStream(topologyBuilder, properties);
final CountDownLatch latch = new CountDownLatch(1);
Runtime.getRuntime().addShutdownHook(new Thread("wordcount-shutdown-hook") {
@Override
public void run() {
rocketMQStream.stop();
latch.countDown();
}
});
try {
rocketMQStream.start();
latch.await();
} catch (final Throwable e) {
System.exit(1);
}
System.exit(0);
}
```
================================================
FILE: build_without_test.sh
================================================
#!/bin/sh
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License ato
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
mvn clean package -Dmaven.test.skip=true
================================================
FILE: core/pom.xml
================================================
rocketmq-streams-allorg.apache.rocketmq1.1.2-SNAPSHOT4.0.0rocketmq-streamsjar88org.apache.rocketmqrocketmq-common${rocketmq.version}org.apache.rocketmqrocketmq-client${rocketmq.version}org.apache.rocketmqrocketmq-toolscom.google.protobufprotobuf-java${rocketmq.version}org.rocksdbrocksdbjni7.6.0commons-iocommons-io2.11.0com.fasterxml.jackson.corejackson-databind2.13.4.2
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/RocketMQStream.java
================================================
package org.apache.rocketmq.streams.core;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.rocketmq.streams.core.common.Constant;
import org.apache.rocketmq.streams.core.exception.RStreamsException;
import org.apache.rocketmq.streams.core.metadata.StreamConfig;
import org.apache.rocketmq.streams.core.running.WorkerThread;
import org.apache.rocketmq.streams.core.topology.TopologyBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
public class RocketMQStream {
private static final Logger logger = LoggerFactory.getLogger(RocketMQStream.class.getName());
private static final AtomicInteger index = new AtomicInteger(1);
private final TopologyBuilder topologyBuilder;
private final Properties properties;
private final List workerThreads = new ArrayList<>();
private final AtomicBoolean started = new AtomicBoolean(false);
private final ScheduledExecutorService executor = Executors.newScheduledThreadPool(
StreamConfig.SCHEDULED_THREAD_NUM,
r -> new Thread(r, "ScanIdleWindowThread_" + index.getAndIncrement()));
public RocketMQStream(TopologyBuilder topologyBuilder, Properties properties) {
this.topologyBuilder = topologyBuilder;
this.properties = properties;
}
public synchronized void start() {
String jobId = topologyBuilder.getJobId();
if (started.get()) {
logger.info("RocketMQStream has been started, jobId=[{}].", jobId);
return;
}
this.started.compareAndSet(false, true);
//启动线程
try {
int threadNum = StreamConfig.STREAMS_PARALLEL_THREAD_NUM;
for (int i = 0; i < threadNum; i++) {
String threadName = String.join("_", Constant.WORKER_THREAD_NAME, jobId, String.valueOf(i));
WorkerThread thread = new WorkerThread(threadName, topologyBuilder, this.properties, executor);
thread.start();
workerThreads.add(thread);
}
} catch (Throwable t) {
logger.error("start RocketMQStream error, jobId=[{}].", jobId, t);
throw new RStreamsException(t);
}
}
public void stop() {
for (WorkerThread thread : workerThreads) {
thread.shutdown();
}
workerThreads.clear();
this.started.set(false);
}
public boolean isRunning() {
return this.started.get();
}
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/common/Constant.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.common;
public class Constant {
public static final String SHUFFLE_KEY_CLASS_NAME = "shuffle.key.class.name";
public static final String SHUFFLE_VALUE_CLASS_NAME = "shuffle.value.class.name";
public final static String STATE_TOPIC_SUFFIX = "-stateTopic";
public static final String SHUFFLE_TOPIC_SUFFIX = "-shuffleTopic";
public static final String SKIP_DATA_ERROR = "skip_data_error";
public static final String SPLIT = "@";
public static final String EMPTY_BODY = "empty_body";
public static final String TRUE = "true";
public static final String SOURCE_TIMESTAMP = "source_timestamp";
public static final String STREAM_TAG = "stream_tag";
public static final String WINDOW_START_TIME = "window_start_time";
public static final String WINDOW_END_TIME = "window_end_time";
public static final String WORKER_THREAD_NAME = "ROCKETMQ_STREAMS";
public static final String STATIC_TOPIC_BROKER_NAME = "__syslo__global__";
public static final String WATERMARK_KEY = "watermark_key";
public static final Long DEFAULT_CONSUME_OFFSET = 0L;
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/DataProcessThrowable.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.exception;
public class DataProcessThrowable extends Throwable {
public DataProcessThrowable() {
}
public DataProcessThrowable(String message) {
super(message);
}
public DataProcessThrowable(String message, Throwable cause) {
super(message, cause);
}
public DataProcessThrowable(Throwable cause) {
super(cause);
}
public DataProcessThrowable(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
super(message, cause, enableSuppression, writableStackTrace);
}
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/DeserializeThrowable.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.exception;
public class DeserializeThrowable extends Throwable {
private static final long serialVersionUID = 2154421351264920776L;
public DeserializeThrowable() {
}
public DeserializeThrowable(String message) {
super(message);
}
public DeserializeThrowable(String message, Throwable cause) {
super(message, cause);
}
public DeserializeThrowable(Throwable cause) {
super(cause);
}
public DeserializeThrowable(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
super(message, cause, enableSuppression, writableStackTrace);
}
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/RStreamsException.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.exception;
public class RStreamsException extends RuntimeException {
private static final long serialVersionUID = 6729806497659471678L;
public RStreamsException() {
}
public RStreamsException(String message) {
super(message);
}
public RStreamsException(String message, Throwable cause) {
super(message, cause);
}
public RStreamsException(Throwable cause) {
super(cause);
}
public RStreamsException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
super(message, cause, enableSuppression, writableStackTrace);
}
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/exception/RecoverStateStoreThrowable.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.exception;
public class RecoverStateStoreThrowable extends Throwable {
public RecoverStateStoreThrowable() {
}
public RecoverStateStoreThrowable(String message) {
super(message);
}
public RecoverStateStoreThrowable(String message, Throwable cause) {
super(message, cause);
}
public RecoverStateStoreThrowable(Throwable cause) {
super(cause);
}
public RecoverStateStoreThrowable(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
super(message, cause, enableSuppression, writableStackTrace);
}
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/AggregateAction.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function;
public interface AggregateAction {
OUT calculate(K key, V value, OUT accumulator);
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/FilterAction.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function;
@FunctionalInterface
public interface FilterAction {
/**
*
* @param value the target to filter
* @return pass if true;
*/
boolean apply(final T value);
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/ForeachAction.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function;
@FunctionalInterface
public interface ForeachAction {
void apply(T value);
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/SelectAction.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function;
@FunctionalInterface
public interface SelectAction {
OUT select(IN value);
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/ValueJoinAction.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function;
public interface ValueJoinAction {
OUT apply(final V1 value1, final V2 value2);
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/ValueMapperAction.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function;
public interface ValueMapperAction {
O convert(final T value) throws Throwable;
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/Accumulator.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function.accumulator;
import java.util.Properties;
//因为需要序列化/反序列化这个类,所以必须给field生成setter/getter方法
public interface Accumulator {
void addValue(V value);
void merge(Accumulator other);
/**
* invoked after the window fired.
*
* @param context the attached properties after window fired.
* @return the value.
*/
R result(Properties context);
Accumulator clone();
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/AvgAccumulator.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function.accumulator;
import java.util.Properties;
public class AvgAccumulator implements Accumulator {
private Double avg;
private Integer num;
@Override
public void addValue(V value) {
if (value instanceof Number) {
Number number = (Number) value;
Double valueToDouble = number.doubleValue();
if (avg == null) {
avg = valueToDouble;
num = 1;
} else {
avg = avg + (valueToDouble - avg) / (num + 1);
num++;
}
} else {
throw new IllegalArgumentException("Calculate avg, input is not a number. value=" + value);
}
}
@Override
public void merge(Accumulator other) {
if (other instanceof AvgAccumulator) {
AvgAccumulator otherAvgAccumulator = (AvgAccumulator) other;
Integer numOther = otherAvgAccumulator.getNum();
Double avgOther = otherAvgAccumulator.getAvg();
avg = avg + numOther / (num + numOther) * (avgOther - avg);
num = num + numOther;
} else {
throw new IllegalArgumentException("Merge avg, input is not a AvgAccumulator.");
}
}
@Override
public Double result(Properties context) {
return avg;
}
@Override
public Accumulator clone() {
return new AvgAccumulator<>();
}
public Double getAvg() {
return avg;
}
public void setAvg(Double avg) {
this.avg = avg;
}
public Integer getNum() {
return num;
}
public void setNum(Integer num) {
this.num = num;
}
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/CountAccumulator.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function.accumulator;
import java.util.Properties;
//因为需要序列化/反序列化这个类,所以必须给field生成setter/getter方法
public class CountAccumulator implements Accumulator {
private Integer count = 0;
@Override
public void addValue(V value) {
count += 1;
}
@Override
public void merge(Accumulator other) {
count += other.result(null);
}
@Override
public Integer result(Properties context) {
return count;
}
public Integer getCount() {
return count;
}
public void setCount(Integer count) {
this.count = count;
}
@Override
public Accumulator clone() {
CountAccumulator accumulator = new CountAccumulator<>();
accumulator.count = this.count;
return accumulator;
}
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/accumulator/MinAccumulator.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function.accumulator;
import java.util.Properties;
public class MinAccumulator implements Accumulator {
private Number min;
@Override
public void addValue(V value) {
if (value instanceof Number) {
Number number = (Number) value;
if (min == null) {
min = number;
} else {
min = Math.min(min.doubleValue(), number.doubleValue());
}
} else {
throw new IllegalArgumentException("min but not a number. value=" + value);
}
}
@Override
public void merge(Accumulator other) {
min = Math.min(min.doubleValue(), other.result(null));
}
@Override
public Double result(Properties context) {
return min.doubleValue();
}
@Override
public Accumulator clone() {
return new MinAccumulator<>();
}
public Number getMin() {
return min;
}
public void setMin(Number min) {
this.min = min;
}
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AccumulatorSupplier.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function.supplier;
import org.apache.rocketmq.common.message.MessageQueue;
import org.apache.rocketmq.streams.core.common.Constant;
import org.apache.rocketmq.streams.core.exception.RecoverStateStoreThrowable;
import org.apache.rocketmq.streams.core.function.accumulator.Accumulator;
import org.apache.rocketmq.streams.core.function.SelectAction;
import org.apache.rocketmq.streams.core.metadata.Data;
import org.apache.rocketmq.streams.core.running.AbstractProcessor;
import org.apache.rocketmq.streams.core.running.Processor;
import org.apache.rocketmq.streams.core.running.StreamContext;
import org.apache.rocketmq.streams.core.state.StateStore;
import org.apache.rocketmq.streams.core.util.ColumnFamilyUtil;
import java.util.function.Supplier;
public class AccumulatorSupplier implements Supplier> {
private final String currentName;
private final String parentName;
private SelectAction selectAction;
private Accumulator accumulator;
public AccumulatorSupplier(String currentName, String parentName, SelectAction selectAction, Accumulator accumulator) {
this.currentName = currentName;
this.parentName = parentName;
this.selectAction = selectAction;
this.accumulator = accumulator;
}
@Override
public Processor get() {
return new AccumulatorProcessor(currentName, parentName, selectAction, accumulator);
}
private class AccumulatorProcessor extends AbstractProcessor {
private final String currentName;
private final String parentName;
private StateStore stateStore;
private MessageQueue stateTopicMessageQueue;
private SelectAction selectAction;
private Accumulator accumulator;
public AccumulatorProcessor(String currentName, String parentName, SelectAction selectAction, Accumulator accumulator) {
this.currentName = currentName;
this.parentName = parentName;
this.selectAction = selectAction;
this.accumulator = accumulator;
}
@Override
public void preProcess(StreamContext context) throws RecoverStateStoreThrowable {
super.preProcess(context);
this.stateStore = super.waitStateReplay();
String stateTopicName = context.getSourceTopic() + Constant.STATE_TOPIC_SUFFIX;
this.stateTopicMessageQueue = new MessageQueue(stateTopicName, context.getSourceBrokerName(), context.getSourceQueueId());
}
@Override
public void process(V data) throws Throwable {
K key = this.context.getKey();
Accumulator value;
byte[] keyBytes = super.object2Byte(key);
byte[] valueBytes = stateStore.get(ColumnFamilyUtil.VALUE_STATE_CF, keyBytes);
if (valueBytes == null || valueBytes.length == 0) {
value = accumulator.clone();
} else {
value = super.byte2Object(valueBytes);
}
R select = selectAction.select(data);
value.addValue(select);
OV result = value.result(null);
byte[] newValueBytes = super.object2Byte(value);
stateStore.put(this.stateTopicMessageQueue, ColumnFamilyUtil.VALUE_STATE_CF, keyBytes, newValueBytes);
Data temp = new Data<>(key, result, this.context.getDataTime(), this.context.getHeader());
Data convert = super.convert(temp);
this.context.forward(convert);
}
}
}
================================================
FILE: core/src/main/java/org/apache/rocketmq/streams/core/function/supplier/AddTagSupplier.java
================================================
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.streams.core.function.supplier;
import org.apache.rocketmq.streams.core.common.Constant;
import org.apache.rocketmq.streams.core.metadata.Data;
import org.apache.rocketmq.streams.core.running.AbstractProcessor;
import org.apache.rocketmq.streams.core.running.Processor;
import java.util.Properties;
import java.util.function.Supplier;
public class AddTagSupplier implements Supplier> {
private final Properties properties = new Properties();
public AddTagSupplier(Supplier