Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[ISSUE #95] Fix source can not consume new queue's messages when topic queue expansion #96

Merged
merged 1 commit into from
Jan 10, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@

import java.lang.management.ManagementFactory;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
Expand Down Expand Up @@ -536,14 +537,25 @@ public void close() throws Exception {
}
}

public void initOffsetTableFromRestoredOffsets(List<MessageQueue> messageQueues) {
public void initOffsetTableFromRestoredOffsets(List<MessageQueue> messageQueues) throws MQClientException {
Preconditions.checkNotNull(restoredOffsets, "restoredOffsets can't be null");
restoredOffsets.forEach(
(mq, offset) -> {
if (messageQueues.contains(mq)) {
offsetTable.put(mq, offset);
}
});

List<MessageQueue> extMessageQueue = new ArrayList<>();
for (MessageQueue messageQueue : messageQueues) {
if (!offsetTable.containsKey(messageQueue)) {
extMessageQueue.add(messageQueue);
}
}
if (extMessageQueue.size() != 0) {
log.info("no restoredOffsets for {}, so init offset for these queues", extMessageQueue);
initOffsets(extMessageQueue);
}
log.info("init offset table [{}] from restoredOffsets successful.", offsetTable);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,16 @@
import org.apache.flink.connector.rocketmq.legacy.common.serialization.SimpleStringDeserializationSchema;
import org.apache.flink.connector.rocketmq.legacy.common.util.TestUtils;

import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
import org.apache.rocketmq.common.message.MessageQueue;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;

import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap;
Expand Down Expand Up @@ -69,6 +73,8 @@ public void testSetStartupMode() throws NoSuchFieldException, IllegalAccessExcep

@Test
public void testRestartFromCheckpoint() throws Exception {
DefaultLitePullConsumer consumer = Mockito.mock(DefaultLitePullConsumer.class);
Mockito.when(consumer.committed(Mockito.any())).thenReturn(40L);
Properties properties = new Properties();
properties.setProperty(RocketMQConfig.CONSUMER_GROUP, "${ConsumerGroup}");
properties.setProperty(RocketMQConfig.CONSUMER_TOPIC, "${SourceTopic}");
Expand All @@ -82,13 +88,19 @@ public void testRestartFromCheckpoint() throws Exception {
map.put(new MessageQueue("tpc", "broker-0", 1), 21L);
map.put(new MessageQueue("tpc", "broker-1", 0), 30L);
map.put(new MessageQueue("tpc", "broker-1", 1), 31L);
List<MessageQueue> allocateMessageQueues = new ArrayList<>(map.keySet());
MessageQueue newMessageQueue = new MessageQueue("tpc", "broker-2", 0);
allocateMessageQueues.add(newMessageQueue);
TestUtils.setFieldValue(source, "messageQueues", allocateMessageQueues);
TestUtils.setFieldValue(source, "consumer", consumer);
TestUtils.setFieldValue(source, "restoredOffsets", map);
TestUtils.setFieldValue(source, "offsetTable", new ConcurrentHashMap<>());
source.initOffsetTableFromRestoredOffsets(new ArrayList<>(map.keySet()));
source.initOffsetTableFromRestoredOffsets(allocateMessageQueues);
Map<MessageQueue, Long> offsetTable = (Map) TestUtils.getFieldValue(source, "offsetTable");
for (Map.Entry<MessageQueue, Long> entry : map.entrySet()) {
assertEquals(offsetTable.containsKey(entry.getKey()), true);
assertEquals(offsetTable.containsValue(entry.getValue()), true);
}
assertEquals(offsetTable.containsKey(newMessageQueue), true);
}
}