2121package com .dtstack .flink .sql .util ;
2222
2323
24- import com .dtstack .flink .sql .classloader .ClassLoaderManager ;
2524import com .dtstack .flink .sql .constrant .ConfigConstrant ;
2625import org .apache .commons .lang3 .StringUtils ;
26+ import org .apache .flink .api .common .time .Time ;
2727import org .apache .flink .api .common .typeinfo .TypeInformation ;
2828import org .apache .flink .runtime .state .filesystem .FsStateBackend ;
2929import org .apache .flink .streaming .api .CheckpointingMode ;
3030import org .apache .flink .streaming .api .TimeCharacteristic ;
3131import org .apache .flink .streaming .api .environment .CheckpointConfig ;
3232import org .apache .flink .streaming .api .environment .StreamExecutionEnvironment ;
33+ import org .apache .flink .table .api .StreamQueryConfig ;
3334import org .apache .flink .table .api .TableEnvironment ;
3435import org .apache .flink .table .api .java .BatchTableEnvironment ;
3536import org .apache .flink .table .api .java .StreamTableEnvironment ;
4748import java .net .URLClassLoader ;
4849import java .util .List ;
4950import java .util .Properties ;
51+ import java .util .regex .Matcher ;
52+ import java .util .regex .Pattern ;
5053
5154/**
5255 * Reason:
@@ -59,6 +62,9 @@ public class FlinkUtil {
5962
6063 private static final Logger logger = LoggerFactory .getLogger (FlinkUtil .class );
6164
65+ private static final String TTL_PATTERN_STR = "^+?([1-9][0-9]*)([dDhHmMsS])$" ;
66+ private static final Pattern TTL_PATTERN = Pattern .compile (TTL_PATTERN_STR );
67+
6268 /**
6369 * 开启checkpoint
6470 * @param env
@@ -257,6 +263,68 @@ public static int getEnvParallelism(Properties properties){
257263 return StringUtils .isNotBlank (parallelismStr )?Integer .parseInt (parallelismStr ):1 ;
258264 }
259265
266+ /**
267+ * 设置ttl
268+ * @param properties
269+ * @param tableEnv
270+ * @return
271+ */
272+ public static void setTableEnvTTL (Properties properties , StreamTableEnvironment tableEnv ) {
273+ String ttlMintimeStr = properties .getProperty (ConfigConstrant .SQL_TTL_MINTIME );
274+ String ttlMaxtimeStr = properties .getProperty (ConfigConstrant .SQL_TTL_MAXTIME );
275+ if (StringUtils .isNotEmpty (ttlMintimeStr ) || StringUtils .isNotEmpty (ttlMaxtimeStr )) {
276+ verityTtl (ttlMintimeStr , ttlMaxtimeStr );
277+ Matcher ttlMintimeStrMatcher = TTL_PATTERN .matcher (ttlMintimeStr );
278+ Matcher ttlMaxtimeStrMatcher = TTL_PATTERN .matcher (ttlMaxtimeStr );
279+
280+ Long ttlMintime = 0L ;
281+ Long ttlMaxtime = 0L ;
282+ if (ttlMintimeStrMatcher .find ()) {
283+ ttlMintime = getTtlTime (Integer .parseInt (ttlMintimeStrMatcher .group (1 )), ttlMintimeStrMatcher .group (2 ));
284+ }
285+ if (ttlMaxtimeStrMatcher .find ()) {
286+ ttlMaxtime = getTtlTime (Integer .parseInt (ttlMaxtimeStrMatcher .group (1 )), ttlMaxtimeStrMatcher .group (2 ));
287+ }
288+ if (0L != ttlMintime && 0L != ttlMaxtime ) {
289+ StreamQueryConfig qConfig = tableEnv .queryConfig ();
290+ qConfig .withIdleStateRetentionTime (Time .milliseconds (ttlMintime ), Time .milliseconds (ttlMaxtime ));
291+ }
292+ }
293+ }
294+
295+ /**
296+ * ttl 校验
297+ * @param ttlMintimeStr 最小时间
298+ * @param ttlMaxtimeStr 最大时间
299+ */
300+ private static void verityTtl (String ttlMintimeStr , String ttlMaxtimeStr ) {
301+ if (null == ttlMintimeStr
302+ || null == ttlMaxtimeStr
303+ || !TTL_PATTERN .matcher (ttlMintimeStr ).find ()
304+ || !TTL_PATTERN .matcher (ttlMaxtimeStr ).find ()) {
305+ throw new RuntimeException ("sql.ttl.min 、sql.ttl.max must be set at the same time . example sql.ttl.min=1h,sql.ttl.max=2h" );
306+ }
307+ }
308+
309+ /**
310+ * 不同单位时间到毫秒的转换
311+ * @param timeNumber 时间值,如:30
312+ * @param timeUnit 单位,d:天,h:小时,m:分,s:秒
313+ * @return
314+ */
315+ private static Long getTtlTime (Integer timeNumber ,String timeUnit ) {
316+ if (timeUnit .equalsIgnoreCase ("d" )) {
317+ return timeNumber * 1000l * 60 * 60 * 24 ;
318+ } else if (timeUnit .equalsIgnoreCase ("h" )) {
319+ return timeNumber * 1000l * 60 * 60 ;
320+ } else if (timeUnit .equalsIgnoreCase ("m" )) {
321+ return timeNumber * 1000l * 60 ;
322+ } else if (timeUnit .equalsIgnoreCase ("s" )) {
323+ return timeNumber * 1000l ;
324+ } else {
325+ throw new RuntimeException ("not support " +timeNumber +timeUnit );
326+ }
327+ }
260328
261329 /**
262330 * 最大并发度
0 commit comments