2222
2323import com .dtstack .flink .sql .enums .ECacheContentType ;
2424import com .dtstack .flink .sql .enums .ECacheType ;
25+ import com .dtstack .flink .sql .factory .DTThreadFactory ;
2526import com .dtstack .flink .sql .metric .MetricConstant ;
2627import com .dtstack .flink .sql .side .cache .AbstractSideCache ;
2728import 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