diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/PatternCache.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/PatternCache.java new file mode 100644 index 00000000000..0b3fe89f4a3 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/PatternCache.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.utils; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.regex.Pattern; + +/** + * LRU cache for compiled {@link Pattern} instances. + * + *

It avoids repeated {@link Pattern#compile(String)} calls for the same regex. + * + *

Example: + * + *

{@code
+ * Pattern pattern = PatternCache.getPattern("aia_t_icc_jjdb_\\d{6}");
+ * Matcher matcher = pattern.matcher(tableName);
+ * }
+ */ +public class PatternCache { + + /** Maximum number of cached patterns. */ + public static final int MAX_CACHE_SIZE = 100; + + /** Pattern cache with LRU eviction. */ + private static final Map CACHE = + new LinkedHashMap(16, 0.75f, true) { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + return size() > MAX_CACHE_SIZE; + } + }; + + /** + * Returns a cached {@link Pattern} or compiles and caches it. + * + * @param regex regular expression + * @return compiled {@link Pattern} + */ + public static synchronized Pattern getPattern(String regex) { + return CACHE.computeIfAbsent(regex, Pattern::compile); + } + + /** Clears the cache (mainly for tests). */ + public static synchronized void clear() { + CACHE.clear(); + } + + /** Returns the current cache size. */ + public static synchronized int size() { + return CACHE.size(); + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/Predicates.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/Predicates.java index f7b8ba9515b..c4b2a90b126 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/Predicates.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/Predicates.java @@ -61,7 +61,17 @@ public static Predicate includes(String regexPatterns) { } public static Set setOfRegex(String input, int regexFlags) { - return setOf(input, RegExSplitterByComma::split, (str) -> Pattern.compile(str, regexFlags)); + return setOf( + input, + RegExSplitterByComma::split, + (str) -> { + // Use PatternCache to avoid recompiling patterns. + if (regexFlags == 0) { + return PatternCache.getPattern(str); + } + // Patterns with flags are not cached yet. + return Pattern.compile(str, regexFlags); + }); } public static Set setOf( diff --git a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/PatternCacheTest.java b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/PatternCacheTest.java new file mode 100644 index 00000000000..210587bc15f --- /dev/null +++ b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/PatternCacheTest.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.utils; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +import java.lang.reflect.Field; +import java.util.regex.Pattern; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Unit tests for {@link PatternCache}. */ +class PatternCacheTest { + + @AfterEach + void tearDown() { + PatternCache.clear(); + } + + @Test + void testCacheHit() { + String regex = "aia_t_icc_jjdb_\\d{6}"; + + Pattern pattern1 = PatternCache.getPattern(regex); + Pattern pattern2 = PatternCache.getPattern(regex); + + // Should return the same instance. + assertThat(pattern1).isSameAs(pattern2); + assertThat(PatternCache.size()).isEqualTo(1); + } + + @Test + void testMultiplePatterns() { + Pattern pattern1 = PatternCache.getPattern("regex1"); + Pattern pattern2 = PatternCache.getPattern("regex2"); + Pattern pattern3 = PatternCache.getPattern("regex1"); + + assertThat(pattern1).isSameAs(pattern3); + assertThat(pattern1).isNotSameAs(pattern2); + assertThat(PatternCache.size()).isEqualTo(2); + } + + @Test + void testLruEvictionWhenCacheExceedsMaxSize() throws Exception { + int maxCacheSize = +PatternCache.MAX_CACHE_SIZE; + + Pattern pattern0 = PatternCache.getPattern("regex0"); + Pattern pattern1 = PatternCache.getPattern("regex1"); + for (int i = 2; i < maxCacheSize; i++) { + PatternCache.getPattern("regex" + i); + } + assertThat(PatternCache.size()).isEqualTo(maxCacheSize); + + assertThat(PatternCache.getPattern("regex0")).isSameAs(pattern0); + + PatternCache.getPattern("regex" + maxCacheSize); + + assertThat(PatternCache.size()).isEqualTo(maxCacheSize); + assertThat(PatternCache.getPattern("regex0")).isSameAs(pattern0); + assertThat(PatternCache.getPattern("regex1")).isNotSameAs(pattern1); + } + + @Test + void testClear() { + PatternCache.getPattern("regex1"); + PatternCache.getPattern("regex2"); + assertThat(PatternCache.size()).isEqualTo(2); + + PatternCache.clear(); + assertThat(PatternCache.size()).isEqualTo(0); + } + + @Test + void testPatternMatching() { + Pattern pattern = PatternCache.getPattern("aia_t_icc_jjdb_\\d{6}"); + + assertThat(pattern.matcher("aia_t_icc_jjdb_202512").matches()).isTrue(); + assertThat(pattern.matcher("aia_t_icc_jjdb_abc").matches()).isFalse(); + } +}