By Ni Ze, RocketMQ Senior Contributor, RocketMQ-Streams Maintainer, and Alibaba Cloud Technical Expert
RocketMQ-Streams is a lightweight stream processing engine. Applications are embedded and started in the form of SDK to perform stream processing and computing. It does not depend on other components and can be deployed with a minimum of 1 core and 1 GB of memory. It has great advantages in resource-sensitive scenarios. It also supports multiple computing types: UTF, UTAF, and UTDF. It has been widely used in security, risk control, edge computing, and other scenarios.
This article will analyze the RocketMQ-Streams construction and data forwarding procedure from the perspective of source code. It also discusses how RocketMQ-Streams implements crash recovery and scaling.
Sample code:
public class RocketMQWindowExample {
public static void main(String[] args) {
DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline");
source.fromRocketmq(
"topicName",
"groupName",
false,
"namesrvAddr")
.map(message -> JSONObject.parseObject((String) message))
.window(TumblingWindow.of(Time.seconds(10)))
.groupBy("groupByKey")
.sum ("field name", "output alias")
.count("total")
.waterMark(5)
.setLocalStorageOnly(true)
.toDataSteam()
.toPrint(1)
.start();
}
}
pom file dependency:
<dependency>
<groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-streams-clients</artifactId>
<version>1.0.1-preview</version>
</dependency>
The preceding code is a simple example. Its main function is to specify the topic from RocketMQ to read the data. After converting the data into JSON format, group it by groupByKey field, launch a new window every ten seconds, accumulate the OutFlow field values, output the results to the total field, and print them to the console. In the calculation above, the input is also allowed to be out of order for five seconds, which means the computation will not be triggered immediately when the window time is up, but the window will wait for five seconds before starting the computation. If the window data arrives within this period, it is still valid. The preceding setLocalStorageOnly is true, which indicates the state is not remotely stored, and only RocksDB is used for local storage. Currently, the RocketMQ-Streams version 1.0.1 still uses MySQL as the remote state store. The next version will use RocketMQ as the remote state store.
As a lightweight stream processing engine, RocketMQ-Streams is essentially used as a client of RocketMQ to consume data. A stream processing instance can process multiple queues, while a queue can only be consumed by one instance. Several RocketMQ-Streams instances form a consumer group to consume data together. You can scale out the instances to increase the consumption of processing capacity. If you reduce the number of instances, rebalance will occur. The consumed queue is automatically rebalanced to other consumer instances. We can also see from the figure above that the compute instances do not directly exchange any data with each other, and all computing processing can be completed independently. This architecture simplifies the design of the RocketMQ-Streams and allows instance scaling.
The processor topology defines the computational logic of the stream processing process for the application, which consists of a series of processor nodes and data flows. For example, the entire processing topology in the sample code at the beginning consists of processing nodes (such as source, map, groupBy, sum, count, and print). There are two special processing nodes:
It does not have any upstream nodes, and it reads data from the outside into RocketMQ-Streams and sends the data downstream for processing.
It does not have any downstream nodes and writes out the processed data to the outside.
The processing topology is only the logical abstraction of stream processing code and will be instantiated when stream computing starts. There is only one computing topology in a stream processing instance for a simple design.
There are two special operators among all stream processing operators: one is the groupBy operator that involves data grouping, and the other is stateful computing (such as count). These two operators affect the construction of the entire computing topology. The following is a specific analysis of how RocketMQ-Streams handles them.
Grouping operator groupBy is special because, after the groupBy operation, subsequent operators expect to operate on data with the same key. For example, after the groupBy ("grade") operation, the sum node is to sum groups according to grade, which requires rerouting data with the same "grade" to a stream compute instance for processing. If not, the results obtained on each instance will be incomplete. The overall output will also be wrong.
RocketMQ-Streams uses the shuffle topic to process. Specifically, the compute instance resends groupBy data back to a RocketMQ topic, selects the target queue according to the key's hash value during the sending process, and then reads the data from this topic for subsequent stream processing. According to the given key and hash algorithm, the same key must be in the same queue, and one queue will only be consumed by one stream processing instance, thus achieving the effect that the same key is routed to one instance for processing.
Stateful operators are opposite to stateless operators. If the computation result is only related to the current input and has nothing to do with the previous input, it is a stateless operator. For example, the results of filter
, map
, and foreach
are only related to the current input. The output result of another operator is related to the current operator and the previous input. For example, the sum
operator requires the summationof the input over a period, which is a stateful operator.
RocketMQ-Streams uses RocksDB as local storage and MySQL as remote storage to store state data. The specific steps are listed below:
The overall data flow diagram is listed below:
Scaling and crash recovery are two sides of the same coin (two expressions of the same thing). If the computing cluster can scale correctly, it can recover from crashes (and vice versa). As we know from the previous introduction, the RocketMQ-Streams has a very good scaling performance. You only need to deploy a new stream compute instance for scale-out and stop the compute instance for scale-in. It is relatively simple for stateless computing. After the scale-out, the previous state is not required for data computing. The scaling of stateful computing involves the migration of states. Stateful scaling can be represented by the following figure:
When compute instances are scaled in from three to two, MQ is reallocated among compute instances with the help of the rebalance of RocketMQ.
MQ2 and MQ3 consumed on Instance 1 are allocated to Instance 2 and Instance 3. The state data of these two MQs needs to be migrated to Instance 2 and Instance 3. This also implies that the state data is stored according to the source data shard, while the scale-out process is the opposite.
In a specific implementation, RocketMQ-Streams uses system messages to trigger the loading and persistence of states.
System message category:
// Add new consumption queue
NewSplitMessage
// Not consuming a queue
RemoveSplitMessage
// The client persists the consumer offset to MQ
CheckPointMessage
When a message is found to come from a new RocketMQ queue (MessageQueue), and RocketMQ-Streams has not processed the message from the queue before, RocketMQ-Streams will send the NewSplitMessage message before the data and pass it through the downstream operator of the processing topology. When the stateful operator receives the message, it will load the state corresponding to the new queue into the local memory RocksDB. When the data arrives, the computation will continue according to this state.
When the compute instance no longer consumes a queue (MessageQueue) after rebalance due to the addition of compute instances or changes in the RocketMQ cluster, the RemoveSplitMessage message is sent. The stateful operator deletes the state in the local RocksDB.
CheckPointMessage is a special system message, and its role is related to the implementation of exactly-once. We need to do exactly-once during scaling to ensure scaling or crash recovery does not affect the computation results. Before the RocketMQ-streams submits the consumer offset to the broker, the CheckPointMessage message is generated and passed to the downstream topology. It will ensure the sink has processed all messages that are about to submit the consumer offset.
Open Source Address
Managed Service Network: Evolution of Application Architecture in Cloud-Native Era
206 posts | 12 followers
FollowAlibaba Cloud Community - November 11, 2022
Alibaba Cloud Native Community - July 19, 2022
Alibaba Cloud Native Community - January 6, 2023
Alibaba Cloud Native Community - July 12, 2022
Alibaba Cloud Community - June 2, 2022
Alibaba Cloud Native Community - March 20, 2023
206 posts | 12 followers
FollowAlibaba Cloud Function Compute is a fully-managed event-driven compute service. It allows you to focus on writing and uploading code without the need to manage infrastructure such as servers.
Learn MoreHigh Performance Computing (HPC) and AI technology helps scientific research institutions to perform viral gene sequencing, conduct new drug research and development, and shorten the research and development cycle.
Learn MoreDeploy custom Alibaba Cloud solutions for business-critical scenarios with Quick Start templates.
Learn MoreElastic and secure virtual cloud servers to cater all your cloud hosting needs.
Learn MoreMore Posts by Alibaba Cloud Native