Skip to content

Commit b7480e1

Browse files
authored
[Fix][connector-rocketmq] commit a correct offset to broker & reduce ThreadInterruptedException log (#6668)
1 parent dba9953 commit b7480e1

File tree

4 files changed

+189
-52
lines changed

4 files changed

+189
-52
lines changed

Diff for: seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqConsumerThread.java

+30
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,10 @@
2323

2424
import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
2525
import org.apache.rocketmq.client.exception.MQClientException;
26+
import org.apache.rocketmq.common.message.MessageQueue;
2627

28+
import java.util.Collections;
29+
import java.util.Objects;
2730
import java.util.concurrent.LinkedBlockingQueue;
2831
import java.util.concurrent.TimeUnit;
2932
import java.util.function.Consumer;
@@ -33,6 +36,11 @@ public class RocketMqConsumerThread implements Runnable {
3336
private final ConsumerMetadata metadata;
3437
private final LinkedBlockingQueue<Consumer<DefaultLitePullConsumer>> tasks;
3538

39+
private MessageQueue assignedMessageQueue;
40+
41+
/** It is different from the committed offset,just means the last offset that has been polled */
42+
private long lastPolledOffset = -2;
43+
3644
public RocketMqConsumerThread(ConsumerMetadata metadata) {
3745
this.metadata = metadata;
3846
this.tasks = new LinkedBlockingQueue<>();
@@ -70,4 +78,26 @@ public void run() {
7078
public LinkedBlockingQueue<Consumer<DefaultLitePullConsumer>> getTasks() {
7179
return tasks;
7280
}
81+
82+
public void assign(RocketMqSourceSplit sourceSplit) throws MQClientException {
83+
boolean messageQueueChanged =
84+
assignedMessageQueue == null
85+
|| !Objects.equals(assignedMessageQueue, sourceSplit.getMessageQueue());
86+
if (messageQueueChanged) {
87+
this.assignedMessageQueue = sourceSplit.getMessageQueue();
88+
consumer.assign(Collections.singleton(assignedMessageQueue));
89+
}
90+
if (messageQueueChanged || lastPolledOffset != sourceSplit.getStartOffset() - 1) {
91+
if (sourceSplit.getStartOffset() >= 0) {
92+
Long committedOffset = consumer.committed(assignedMessageQueue);
93+
if (!Objects.equals(committedOffset, sourceSplit.getStartOffset())) {
94+
consumer.seek(assignedMessageQueue, sourceSplit.getStartOffset());
95+
}
96+
}
97+
}
98+
}
99+
100+
public void markLastPolledOffset(long offset) {
101+
this.lastPolledOffset = offset;
102+
}
73103
}

Diff for: seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java

+42-52
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.seatunnel.connectors.seatunnel.rocketmq.source;
1919

2020
import org.apache.seatunnel.shade.com.google.common.collect.Maps;
21-
import org.apache.seatunnel.shade.com.google.common.collect.Sets;
2221

2322
import org.apache.seatunnel.api.serialization.DeserializationSchema;
2423
import org.apache.seatunnel.api.source.Boundedness;
@@ -37,6 +36,7 @@
3736
import java.util.HashSet;
3837
import java.util.List;
3938
import java.util.Map;
39+
import java.util.Objects;
4040
import java.util.Set;
4141
import java.util.concurrent.CompletableFuture;
4242
import java.util.concurrent.ConcurrentHashMap;
@@ -50,7 +50,7 @@ public class RocketMqSourceReader implements SourceReader<SeaTunnelRow, RocketMq
5050

5151
private static final long THREAD_WAIT_TIME = 500L;
5252

53-
private final SourceReader.Context context;
53+
private final Context context;
5454
private final ConsumerMetadata metadata;
5555
private final Set<RocketMqSourceSplit> sourceSplits;
5656
private final Map<Long, Map<MessageQueue, Long>> checkpointOffsets;
@@ -65,7 +65,7 @@ public class RocketMqSourceReader implements SourceReader<SeaTunnelRow, RocketMq
6565
public RocketMqSourceReader(
6666
ConsumerMetadata metadata,
6767
DeserializationSchema<SeaTunnelRow> deserializationSchema,
68-
SourceReader.Context context) {
68+
Context context) {
6969
this.metadata = metadata;
7070
this.context = context;
7171
this.sourceSplits = new HashSet<>();
@@ -115,21 +115,16 @@ public void pollNext(Collector<SeaTunnelRow> output) throws Exception {
115115
sourceSplit -> {
116116
CompletableFuture<Void> completableFuture = new CompletableFuture<>();
117117
try {
118-
consumerThreads
119-
.get(sourceSplit.getMessageQueue())
118+
RocketMqConsumerThread rocketMqConsumerThread =
119+
consumerThreads.get(sourceSplit.getMessageQueue());
120+
rocketMqConsumerThread
120121
.getTasks()
121122
.put(
122123
consumer -> {
123124
try {
124-
Set<MessageQueue> messageQueues =
125-
Sets.newHashSet(
126-
sourceSplit.getMessageQueue());
127-
consumer.assign(messageQueues);
128-
if (sourceSplit.getStartOffset() >= 0) {
129-
consumer.seek(
130-
sourceSplit.getMessageQueue(),
131-
sourceSplit.getStartOffset());
132-
}
125+
rocketMqConsumerThread.assign(sourceSplit);
126+
MessageQueue assignedMessageQueue =
127+
sourceSplit.getMessageQueue();
133128
List<MessageExt> records =
134129
consumer.poll(
135130
metadata.getBaseConfig()
@@ -141,47 +136,36 @@ public void pollNext(Collector<SeaTunnelRow> output) throws Exception {
141136
sourceSplit.getStartOffset(),
142137
sourceSplit.getEndOffset());
143138
}
144-
Map<MessageQueue, List<MessageExt>> groupRecords =
139+
List<MessageExt> messages =
145140
records.stream()
146-
.collect(
147-
Collectors.groupingBy(
148-
record ->
149-
new MessageQueue(
150-
record
151-
.getTopic(),
152-
record
153-
.getBrokerName(),
154-
record
155-
.getQueueId())));
156-
for (MessageQueue messageQueue : messageQueues) {
157-
if (!groupRecords.containsKey(messageQueue)) {
158-
continue;
159-
}
160-
List<MessageExt> messages =
161-
groupRecords.get(messageQueue);
162-
for (MessageExt record : messages) {
163-
deserializationSchema.deserialize(
164-
record.getBody(), output);
165-
if (Boundedness.BOUNDED.equals(
166-
context.getBoundedness())
167-
&& record.getQueueOffset()
168-
>= sourceSplit
169-
.getEndOffset()) {
170-
break;
171-
}
172-
}
173-
long lastOffset = -1;
174-
if (!messages.isEmpty()) {
175-
lastOffset =
176-
messages.get(messages.size() - 1)
177-
.getQueueOffset();
178-
sourceSplit.setStartOffset(lastOffset);
179-
}
180-
181-
if (lastOffset >= sourceSplit.getEndOffset()) {
182-
sourceSplit.setEndOffset(lastOffset);
141+
.filter(
142+
record ->
143+
isQueueMatch(
144+
assignedMessageQueue,
145+
record))
146+
.collect(Collectors.toList());
147+
long lastOffset = -1;
148+
for (MessageExt record : messages) {
149+
deserializationSchema.deserialize(
150+
record.getBody(), output);
151+
lastOffset = record.getQueueOffset();
152+
if (Boundedness.BOUNDED.equals(
153+
context.getBoundedness())
154+
&& record.getQueueOffset()
155+
>= sourceSplit.getEndOffset()) {
156+
break;
183157
}
184158
}
159+
if (lastOffset >= 0) {
160+
// set start offset for next poll cycleLife
161+
sourceSplit.setStartOffset(lastOffset + 1);
162+
rocketMqConsumerThread.markLastPolledOffset(
163+
lastOffset);
164+
}
165+
if (lastOffset >= sourceSplit.getEndOffset()) {
166+
// just for bounded mode
167+
sourceSplit.setEndOffset(lastOffset);
168+
}
185169
} catch (Exception e) {
186170
completableFuture.completeExceptionally(e);
187171
}
@@ -200,6 +184,12 @@ record ->
200184
}
201185
}
202186

187+
private boolean isQueueMatch(MessageQueue assignedMessageQueue, MessageExt record) {
188+
return Objects.equals(assignedMessageQueue.getTopic(), record.getTopic())
189+
&& Objects.equals(assignedMessageQueue.getBrokerName(), record.getBrokerName())
190+
&& Objects.equals(assignedMessageQueue.getQueueId(), record.getQueueId());
191+
}
192+
203193
@Override
204194
public List<RocketMqSourceSplit> snapshotState(long checkpointId) throws Exception {
205195
List<RocketMqSourceSplit> pendingSplit =

Diff for: seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rocketmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rocketmq/RocketMqIT.java

+45
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,9 @@
3737
import org.apache.seatunnel.connectors.seatunnel.rocketmq.serialize.DefaultSeaTunnelRowSerializer;
3838
import org.apache.seatunnel.e2e.common.TestResource;
3939
import org.apache.seatunnel.e2e.common.TestSuiteBase;
40+
import org.apache.seatunnel.e2e.common.container.EngineType;
4041
import org.apache.seatunnel.e2e.common.container.TestContainer;
42+
import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer;
4143
import org.apache.seatunnel.engine.common.Constant;
4244

4345
import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
@@ -68,10 +70,12 @@
6870
import java.time.Duration;
6971
import java.time.LocalDate;
7072
import java.time.LocalDateTime;
73+
import java.util.Arrays;
7174
import java.util.Collections;
7275
import java.util.HashMap;
7376
import java.util.List;
7477
import java.util.Map;
78+
import java.util.Set;
7579
import java.util.UUID;
7680

7781
import static org.apache.seatunnel.e2e.connector.rocketmq.RocketMqContainer.NAMESRV_PORT;
@@ -214,6 +218,27 @@ public void testSourceRocketMqTextToConsole(TestContainer container)
214218
Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr());
215219
}
216220

221+
@TestTemplate
222+
@DisabledOnContainer(
223+
value = {},
224+
type = {EngineType.SPARK, EngineType.FLINK},
225+
disabledReason = "flink and spark won't commit offset when batch job finished")
226+
public void testSourceRocketMqTextToConsoleWithOffsetCheck(TestContainer container)
227+
throws IOException, InterruptedException {
228+
DefaultSeaTunnelRowSerializer serializer =
229+
new DefaultSeaTunnelRowSerializer(
230+
"test_topic_text_offset_check",
231+
SEATUNNEL_ROW_TYPE,
232+
SchemaFormat.TEXT,
233+
DEFAULT_FIELD_DELIMITER);
234+
generateTestData(
235+
row -> serializer.serializeRow(row), "test_topic_text_offset_check", 0, 10);
236+
Container.ExecResult execResult =
237+
container.executeJob("/rocketmq-source_tex_with_offset_check.conf");
238+
Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr());
239+
checkOffsetNoDiff("test_topic_text_offset_check", "SeaTunnel-Consumer-Group");
240+
}
241+
217242
@TestTemplate
218243
public void testSourceRocketMqJsonToConsole(TestContainer container)
219244
throws IOException, InterruptedException {
@@ -375,6 +400,26 @@ private Map<String, String> getRocketMqConsumerData(String topicName) {
375400
return data;
376401
}
377402

403+
private void checkOffsetNoDiff(String topicName, String consumerGroup) {
404+
RocketMqBaseConfiguration config = newConfiguration();
405+
config.setGroupId(consumerGroup);
406+
List<Map<MessageQueue, TopicOffset>> offsetTopics =
407+
RocketMqAdminUtil.offsetTopics(config, Arrays.asList(topicName));
408+
Map<MessageQueue, TopicOffset> offsetMap = offsetTopics.get(0);
409+
Set<MessageQueue> messageQueues = offsetMap.keySet();
410+
Map<MessageQueue, Long> currentOffsets =
411+
RocketMqAdminUtil.currentOffsets(config, Arrays.asList(topicName), messageQueues);
412+
for (Map.Entry<MessageQueue, TopicOffset> offsetEntry : offsetMap.entrySet()) {
413+
MessageQueue messageQueue = offsetEntry.getKey();
414+
long maxOffset = offsetEntry.getValue().getMaxOffset();
415+
Long consumeOffset = currentOffsets.get(messageQueue);
416+
Assertions.assertEquals(
417+
maxOffset,
418+
consumeOffset,
419+
"Offset different,maxOffset=" + maxOffset + ",consumeOffset=" + consumeOffset);
420+
}
421+
}
422+
378423
public RocketMqBaseConfiguration newConfiguration() {
379424
return RocketMqBaseConfiguration.newBuilder()
380425
.groupId(ROCKETMQ_GROUP)
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,72 @@
1+
#
2+
# Licensed to the Apache Software Foundation (ASF) under one or more
3+
# contributor license agreements. See the NOTICE file distributed with
4+
# this work for additional information regarding copyright ownership.
5+
# The ASF licenses this file to You under the Apache License, Version 2.0
6+
# (the "License"); you may not use this file except in compliance with
7+
# the License. You may obtain a copy of the License at
8+
#
9+
# http://www.apache.org/licenses/LICENSE-2.0
10+
#
11+
# Unless required by applicable law or agreed to in writing, software
12+
# distributed under the License is distributed on an "AS IS" BASIS,
13+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
# See the License for the specific language governing permissions and
15+
# limitations under the License.
16+
#
17+
######
18+
###### This config file is a demonstration of streaming processing in seatunnel config
19+
######
20+
21+
env {
22+
parallelism = 1
23+
job.mode = "BATCH"
24+
checkpoint.interval = 1000
25+
26+
# You can set spark configuration here
27+
spark.app.name = "SeaTunnel"
28+
spark.executor.instances = 1
29+
spark.executor.cores = 1
30+
spark.executor.memory = "1g"
31+
spark.master = local
32+
}
33+
34+
source {
35+
Rocketmq {
36+
name.srv.addr = "rocketmq-e2e:9876"
37+
topics = "test_topic_text_offset_check"
38+
result_table_name = "rocketmq_table"
39+
consumer.group = "SeaTunnel-Consumer-Group"
40+
schema = {
41+
fields {
42+
id = bigint
43+
c_map = "map<string, smallint>"
44+
c_array = "array<tinyint>"
45+
c_string = string
46+
c_boolean = boolean
47+
c_tinyint = tinyint
48+
c_smallint = smallint
49+
c_int = int
50+
c_bigint = bigint
51+
c_float = float
52+
c_double = double
53+
c_decimal = "decimal(2, 1)"
54+
c_bytes = bytes
55+
c_date = date
56+
c_timestamp = timestamp
57+
}
58+
}
59+
format = text
60+
# The default field delimiter is ","
61+
field_delimiter = ","
62+
}
63+
}
64+
65+
transform {
66+
}
67+
68+
sink {
69+
Console {
70+
source_table_name = "rocketmq_table"
71+
}
72+
}

0 commit comments

Comments
 (0)