Skip to content

Commit b1b56ad

Browse files
author
dapeng
committed
修复重复创建线程池的bug
1 parent e463694 commit b1b56ad

File tree

1 file changed

+5
-2
lines changed

1 file changed

+5
-2
lines changed

core/src/main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java

+5-2
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222

2323
import com.dtstack.flink.sql.enums.ECacheContentType;
2424
import com.dtstack.flink.sql.enums.ECacheType;
25+
import com.dtstack.flink.sql.factory.DTThreadFactory;
2526
import com.dtstack.flink.sql.metric.MetricConstant;
2627
import com.dtstack.flink.sql.side.cache.AbstractSideCache;
2728
import com.dtstack.flink.sql.side.cache.CacheObj;
@@ -68,6 +69,7 @@ public abstract class BaseAsyncReqRow extends RichAsyncFunction<CRow, CRow> impl
6869
private int timeOutNum = 0;
6970
protected BaseSideInfo sideInfo;
7071
protected transient Counter parseErrorRecords;
72+
private transient ThreadPoolExecutor cancelExecutor;
7173

7274
public BaseAsyncReqRow(BaseSideInfo sideInfo){
7375
this.sideInfo = sideInfo;
@@ -82,6 +84,8 @@ public void open(Configuration parameters) throws Exception {
8284
super.open(parameters);
8385
initCache();
8486
initMetric();
87+
cancelExecutor = new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(100000),
88+
new DTThreadFactory("cancel-timer-executor"));
8589
LOG.info("async dim table config info: {} ", sideInfo.getSideTableInfo().toString());
8690
}
8791

@@ -248,12 +252,11 @@ public void onProcessingTime(long timestamp) throws Exception {
248252
}
249253

250254
protected void cancelTimerWhenComplete(ResultFuture<CRow> resultFuture, ScheduledFuture<?> timerFuture){
251-
ThreadPoolExecutor executors = new ThreadPoolExecutor(1, 1,0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>());
252255
if(resultFuture instanceof StreamRecordQueueEntry){
253256
StreamRecordQueueEntry streamRecordBufferEntry = (StreamRecordQueueEntry) resultFuture;
254257
streamRecordBufferEntry.onComplete((Object value) -> {
255258
timerFuture.cancel(true);
256-
},executors);
259+
}, cancelExecutor);
257260
}
258261
}
259262

0 commit comments

Comments
 (0)