[fix](mtmv)resolve task tvf concurrent modification exception #30497

LinkedList is not thread safe and there may be issues with concurrent queries and writes. Change it to ConcurrentLinkedQueue

Caused by: java.util.ConcurrentModificationException
	at java.util.LinkedList$ListItr.checkForComodification(LinkedList.java:966) ~[?:1.8.0_131]
	at java.util.LinkedList$ListItr.next(LinkedList.java:888) ~[?:1.8.0_131]
	at org.apache.doris.tablefunction.MetadataGenerator.taskMetadataResult(MetadataGenerator.java:694) ~[doris-fe.jar:1.2-SNAPSHOT]
	at org.apache.doris.tablefunction.MetadataGenerator.getMetadataTable(MetadataGenerator.java:119) ~[doris-fe.jar:1.2-SNAPSHOT]
	at org.apache.doris.service.FrontendServiceImpl.fetchSchemaTableData(FrontendServiceImpl.java:2195) ~[doris-fe.jar:1.2-SNAPSHOT]
	... 13 more
This commit is contained in:
zhangdong
2024-01-30 11:55:20 +08:00
committed by yiguolei
parent 53c624ffa0
commit 9e63015e2f
2 changed files with 6 additions and 7 deletions

View File

@ -185,7 +185,7 @@ public class MTMVJob extends AbstractJob<MTMVTask, MTMVTaskContext> {
LOG.warn("get mtmv failed", e);
return Lists.newArrayList();
}
return mtmv.getJobInfo().getHistoryTasks();
return Lists.newArrayList(mtmv.getJobInfo().getHistoryTasks());
}
@Override

View File

@ -20,10 +20,9 @@ package org.apache.doris.mtmv;
import org.apache.doris.common.Config;
import org.apache.doris.job.extensions.mtmv.MTMVTask;
import com.google.common.collect.Lists;
import com.google.gson.annotations.SerializedName;
import java.util.LinkedList;
import java.util.concurrent.ConcurrentLinkedQueue;
/**
* MTMVJobInfo
@ -32,11 +31,11 @@ public class MTMVJobInfo {
@SerializedName("jobName")
private String jobName;
@SerializedName("ht")
private LinkedList<MTMVTask> historyTasks;
private ConcurrentLinkedQueue<MTMVTask> historyTasks;
public MTMVJobInfo(String jobName) {
this.jobName = jobName;
historyTasks = Lists.newLinkedList();
historyTasks = new ConcurrentLinkedQueue<>();
}
public String getJobName() {
@ -49,11 +48,11 @@ public class MTMVJobInfo {
}
historyTasks.add(task);
if (historyTasks.size() > Config.max_persistence_task_count) {
historyTasks.removeFirst();
historyTasks.poll();
}
}
public LinkedList<MTMVTask> getHistoryTasks() {
public ConcurrentLinkedQueue<MTMVTask> getHistoryTasks() {
return historyTasks;
}