Skip to content

Commit b2876ad

Browse files
authored
[INLONG-12073][Manager] Optimize Pulsar message query with thread pool and proper error handling (#12074)
* [INLONG-12073][Manager] Optimize Pulsar message query with thread pool and proper error handling * [INLONG-12073][Manager] Throw exception for query caller * [INLONG-12073][Manager] Add unit tests * [INLONG-12073][Manager] Fix the code formats * [INLONG-12073][Manager] Remove the randomly select method * [INLONG-12073][Manager] Use ConcurrentLinkedQueue to improve performance * [INLONG-12073][Manager] Add name label for manager-schedule module * [INLONG-12073][Manager] Remove unused properties
1 parent 3cf9685 commit b2876ad

8 files changed

Lines changed: 706 additions & 67 deletions

File tree

inlong-manager/manager-schedule/pom.xml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
</parent>
2626

2727
<artifactId>manager-schedule</artifactId>
28+
<name>Apache InLong - Manager Schedule</name>
2829

2930
<properties>
3031
<quartz.version>2.3.2</quartz.version>

inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/QueueResourceOperator.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -83,11 +83,10 @@ default void deleteQueueForStream(InlongGroupInfo groupInfo, InlongStreamInfo st
8383
* @param groupInfo inlong group info
8484
* @param streamInfo inlong stream info
8585
* @param request query message request
86-
* @throws Exception any exception if occurred
8786
* @return query brief mq message info
8887
*/
8988
default List<BriefMQMessage> queryLatestMessages(InlongGroupInfo groupInfo, InlongStreamInfo streamInfo,
90-
QueryMessageRequest request) throws Exception {
89+
QueryMessageRequest request) {
9190
return null;
9291
}
9392

inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/pulsar/PulsarQueueResourceOperator.java

Lines changed: 166 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -44,17 +44,26 @@
4444
import org.apache.inlong.manager.service.stream.InlongStreamService;
4545

4646
import com.google.common.base.Objects;
47+
import com.google.common.util.concurrent.ThreadFactoryBuilder;
4748
import lombok.extern.slf4j.Slf4j;
4849
import org.apache.commons.collections.CollectionUtils;
4950
import org.apache.commons.lang3.StringUtils;
5051
import org.springframework.beans.factory.annotation.Autowired;
52+
import org.springframework.beans.factory.annotation.Value;
5153
import org.springframework.stereotype.Service;
5254

55+
import javax.annotation.PostConstruct;
56+
import javax.annotation.PreDestroy;
57+
5358
import java.util.ArrayList;
5459
import java.util.Collections;
5560
import java.util.List;
56-
import java.util.concurrent.Executors;
57-
import java.util.concurrent.ScheduledExecutorService;
61+
import java.util.concurrent.ConcurrentLinkedQueue;
62+
import java.util.concurrent.ExecutorService;
63+
import java.util.concurrent.Future;
64+
import java.util.concurrent.LinkedBlockingQueue;
65+
import java.util.concurrent.RejectedExecutionException;
66+
import java.util.concurrent.ThreadPoolExecutor;
5867
import java.util.concurrent.TimeUnit;
5968

6069
/**
@@ -80,7 +89,59 @@ public class PulsarQueueResourceOperator implements QueueResourceOperator {
8089
@Autowired
8190
private PulsarOperator pulsarOperator;
8291

83-
private ScheduledExecutorService executor = Executors.newScheduledThreadPool(10);
92+
@Value("${pulsar.query.poolSize:10}")
93+
private int poolSize;
94+
95+
@Value("${pulsar.query.keepAliveSeconds:60}")
96+
private long keepAliveSeconds;
97+
98+
@Value("${pulsar.query.queueCapacity:100}")
99+
private int queueCapacity;
100+
101+
@Value("${pulsar.query.queryTimeoutSeconds:10}")
102+
private int queryTimeoutSeconds;
103+
104+
/**
105+
* Thread pool for querying messages from multiple Pulsar clusters concurrently.
106+
* Configuration is loaded from application properties with prefix 'pulsar.query'.
107+
*/
108+
private ExecutorService messageQueryExecutor;
109+
110+
/**
111+
* Initialize the executor service after bean creation.
112+
*/
113+
@PostConstruct
114+
public void init() {
115+
// Initialize the executor service with same core pool size and max core pool size
116+
this.messageQueryExecutor = new ThreadPoolExecutor(
117+
poolSize,
118+
poolSize,
119+
keepAliveSeconds,
120+
TimeUnit.SECONDS,
121+
new LinkedBlockingQueue<>(queueCapacity),
122+
new ThreadFactoryBuilder().setNameFormat("pulsar-message-query-%d").build(),
123+
// Use AbortPolicy to throw exception when the queue is full
124+
new ThreadPoolExecutor.AbortPolicy());
125+
log.info("Init message query executor, poolSize={}, keepAliveSeconds={}, queueCapacity={}",
126+
poolSize, keepAliveSeconds, queueCapacity);
127+
}
128+
129+
/**
130+
* Shutdown the executor service when the bean is destroyed.
131+
*/
132+
@PreDestroy
133+
public void shutdown() {
134+
log.info("Shutting down pulsar message query executor");
135+
messageQueryExecutor.shutdown();
136+
try {
137+
if (!messageQueryExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
138+
messageQueryExecutor.shutdownNow();
139+
}
140+
} catch (InterruptedException e) {
141+
messageQueryExecutor.shutdownNow();
142+
Thread.currentThread().interrupt();
143+
}
144+
}
84145

85146
@Override
86147
public boolean accept(String mqType) {
@@ -307,34 +368,117 @@ private void deletePulsarTopic(InlongPulsarInfo pulsarInfo, PulsarClusterInfo pu
307368
* Query latest message from pulsar
308369
*/
309370
public List<BriefMQMessage> queryLatestMessages(InlongGroupInfo groupInfo, InlongStreamInfo streamInfo,
310-
QueryMessageRequest request) throws Exception {
311-
List<ClusterInfo> pulsarClusterList = clusterService.listByTagAndType(groupInfo.getInlongClusterTag(),
312-
ClusterType.PULSAR);
313-
List<BriefMQMessage> briefMQMessages = Collections.synchronizedList(new ArrayList<>());
314-
QueryCountDownLatch queryLatch = new QueryCountDownLatch(request.getMessageCount(), pulsarClusterList.size());
371+
QueryMessageRequest request) {
372+
String groupId = streamInfo.getInlongGroupId();
373+
String clusterTag = groupInfo.getInlongClusterTag();
374+
List<ClusterInfo> clusterInfos = clusterService.listByTagAndType(clusterTag, ClusterType.PULSAR);
375+
if (CollectionUtils.isEmpty(clusterInfos)) {
376+
log.warn("No pulsar cluster found for clusterTag={} for groupId={}", clusterTag, groupId);
377+
return Collections.emptyList();
378+
}
379+
380+
// Select clusters and calculate per-cluster query count
381+
Integer requestCount = request.getMessageCount();
382+
int clusterSize = clusterInfos.size();
383+
QueryCountDownLatch queryLatch = new QueryCountDownLatch(requestCount, clusterSize);
384+
log.debug("Query pulsar message in {} clusters, each cluster query {} messages", clusterSize, requestCount);
385+
386+
// Extract common parameters
315387
InlongPulsarInfo inlongPulsarInfo = ((InlongPulsarInfo) groupInfo);
316-
for (ClusterInfo clusterInfo : pulsarClusterList) {
317-
QueryLatestMessagesRunnable task = new QueryLatestMessagesRunnable(inlongPulsarInfo, streamInfo,
318-
(PulsarClusterInfo) clusterInfo, pulsarOperator, request, briefMQMessages, queryLatch);
319-
this.executor.execute(task);
388+
String tenant = inlongPulsarInfo.getPulsarTenant();
389+
String namespace = inlongPulsarInfo.getMqResource();
390+
String topicName = streamInfo.getMqResource();
391+
boolean serialQueue = InlongConstants.PULSAR_QUEUE_TYPE_SERIAL.equals(inlongPulsarInfo.getQueueModule());
392+
393+
// Submit query tasks to thread pool, each task queries from one cluster
394+
// Use submit() instead of execute() to get Future for cancellation support
395+
List<Future<?>> submittedTasks = new ArrayList<>();
396+
// Use ConcurrentLinkedQueue for thread-safe message collection,
397+
// its performance is better than Collections.synchronizedList
398+
ConcurrentLinkedQueue<BriefMQMessage> messageResultQueue = new ConcurrentLinkedQueue<>();
399+
for (ClusterInfo clusterInfo : clusterInfos) {
400+
PulsarClusterInfo pulsarCluster = (PulsarClusterInfo) clusterInfo;
401+
if (StringUtils.isBlank(tenant)) {
402+
tenant = pulsarCluster.getPulsarTenant();
403+
}
404+
String fullTopicName = tenant + "/" + namespace + "/" + topicName;
405+
// Create a copy of request with adjusted message count for this cluster
406+
QueryMessageRequest currentRequest = buildRequestForSingleCluster(request, requestCount);
407+
QueryLatestMessagesRunnable task = new QueryLatestMessagesRunnable(pulsarOperator, streamInfo,
408+
pulsarCluster, serialQueue, fullTopicName, currentRequest, messageResultQueue, queryLatch);
409+
try {
410+
Future<?> future = this.messageQueryExecutor.submit(task);
411+
submittedTasks.add(future);
412+
} catch (RejectedExecutionException e) {
413+
// Cancel all previously submitted tasks before throwing exception
414+
log.error("Failed to submit query task for groupId={}, cancelling {} submitted tasks",
415+
groupId, submittedTasks.size(), e);
416+
cancelSubmittedTasks(submittedTasks);
417+
throw new BusinessException("Query messages task rejected: too many concurrent requests");
418+
}
419+
}
420+
421+
// Wait for tasks to complete with a configurable timeout
422+
String streamId = streamInfo.getInlongStreamId();
423+
try {
424+
boolean completed = queryLatch.await(queryTimeoutSeconds, TimeUnit.SECONDS);
425+
if (!completed) {
426+
log.warn("Query messages timeout for groupId={}, streamId={}, collected {} messages",
427+
groupId, streamId, messageResultQueue.size());
428+
}
429+
} catch (InterruptedException e) {
430+
throw new BusinessException(String.format("Query messages task interrupted for groupId=%s, streamId=%s",
431+
groupId, streamId));
432+
}
433+
434+
log.info("Success query pulsar message for groupId={}, streamId={}", groupId, streamId);
435+
List<BriefMQMessage> messageResultList = new ArrayList<>(messageResultQueue);
436+
437+
// if query result size is less than request count, return all, otherwise truncate to request count
438+
if (messageResultList.isEmpty() || messageResultList.size() <= requestCount) {
439+
return messageResultList;
320440
}
321-
queryLatch.await(30, TimeUnit.SECONDS);
322-
log.info("success query pulsar message for groupId={}, streamId={}", streamInfo.getInlongGroupId(),
323-
streamInfo.getInlongStreamId());
324-
325-
int finalMsgCount = Math.min(request.getMessageCount(), briefMQMessages.size());
326-
if (finalMsgCount > 0) {
327-
return new ArrayList<>(briefMQMessages.subList(0, finalMsgCount));
328-
} else {
329-
return new ArrayList<>();
441+
442+
return new ArrayList<>(messageResultList.subList(0, requestCount));
443+
}
444+
445+
/**
446+
* Build a new QueryMessageRequest with adjusted message count for a specific cluster.
447+
*/
448+
private QueryMessageRequest buildRequestForSingleCluster(QueryMessageRequest original, int messageCount) {
449+
return QueryMessageRequest.builder()
450+
.groupId(original.getGroupId())
451+
.streamId(original.getStreamId())
452+
.messageCount(messageCount)
453+
.fieldName(original.getFieldName())
454+
.operationType(original.getOperationType())
455+
.targetValue(original.getTargetValue())
456+
.build();
457+
}
458+
459+
/**
460+
* Cancel all submitted tasks when an error occurs.
461+
* This method attempts to cancel tasks with interrupt flag set to true,
462+
* allowing running tasks to be interrupted if they check for interruption.
463+
*
464+
* @param submittedTasks list of Future objects representing submitted tasks
465+
*/
466+
private void cancelSubmittedTasks(List<java.util.concurrent.Future<?>> submittedTasks) {
467+
int cancelledCount = 0;
468+
for (java.util.concurrent.Future<?> future : submittedTasks) {
469+
// mayInterruptIfRunning=true allows interrupting running tasks
470+
if (future.cancel(true)) {
471+
cancelledCount++;
472+
}
330473
}
474+
log.info("Cancelled {}/{} submitted tasks", cancelledCount, submittedTasks.size());
331475
}
332476

333477
/**
334478
* Reset cursor for consumer group
335479
*/
336480
public void resetCursor(InlongGroupInfo groupInfo, InlongStreamEntity streamEntity, StreamSinkEntity sinkEntity,
337-
Long resetTime) throws Exception {
481+
Long resetTime) {
338482
log.info("begin to reset cursor for sinkId={}", sinkEntity.getId());
339483
InlongPulsarInfo pulsarInfo = (InlongPulsarInfo) groupInfo;
340484
List<ClusterInfo> clusterInfos = clusterService.listByTagAndType(pulsarInfo.getInlongClusterTag(),

inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/pulsar/QueryCountDownLatch.java

Lines changed: 35 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -21,26 +21,55 @@
2121
import java.util.concurrent.TimeUnit;
2222

2323
/**
24-
* QueryCountDownLatch
24+
* QueryCountDownLatch for managing query task and data completion.
25+
* <p>
26+
* This class provides two types of countdown:
27+
* <ul>
28+
* <li>Task countdown: tracks the number of tasks completed (regardless of success or failure)</li>
29+
* <li>Data countdown: tracks the number of data items retrieved</li>
30+
* </ul>
31+
* The flagLatch is released when either all tasks complete or enough data is collected.
2532
*/
2633
public class QueryCountDownLatch {
2734

28-
private CountDownLatch dataLatch;
29-
private CountDownLatch taskLatch;
30-
private CountDownLatch flagLatch;
35+
private final CountDownLatch dataLatch;
36+
private final CountDownLatch taskLatch;
37+
private final CountDownLatch flagLatch;
3138

3239
public QueryCountDownLatch(int dataSize, int taskSize) {
3340
this.dataLatch = new CountDownLatch(dataSize);
3441
this.taskLatch = new CountDownLatch(taskSize);
3542
this.flagLatch = new CountDownLatch(1);
3643
}
3744

38-
public void countDown(int dataDownSize) {
45+
/**
46+
* Called when a task completes (regardless of success or failure).
47+
* This should be called in a finally block to ensure it's always executed.
48+
*/
49+
public void taskCountDown() {
3950
this.taskLatch.countDown();
51+
checkAndRelease();
52+
}
53+
54+
/**
55+
* Called when data items are successfully retrieved.
56+
*
57+
* @param dataDownSize the number of data items retrieved
58+
*/
59+
public void dataCountDown(int dataDownSize) {
4060
for (int i = 0; i < dataDownSize; i++) {
4161
this.dataLatch.countDown();
4262
}
43-
if (this.taskLatch.getCount() == 0 || this.dataLatch.getCount() == 0) {
63+
checkAndRelease();
64+
}
65+
66+
/**
67+
* Check if the flagLatch should be released.
68+
* Release when all tasks complete or enough data is collected.
69+
*/
70+
private synchronized void checkAndRelease() {
71+
if (this.flagLatch.getCount() > 0
72+
&& (this.taskLatch.getCount() == 0 || this.dataLatch.getCount() == 0)) {
4473
this.flagLatch.countDown();
4574
}
4675
}

0 commit comments

Comments
 (0)