Skip to content

Commit d81bd13

Browse files
lordgarrishAndrei Nadyktov
authored andcommitted
IGNITE-22530 Make each regex pattern a single string
1 parent bae4541 commit d81bd13

11 files changed

Lines changed: 79 additions & 72 deletions

modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -70,6 +70,9 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx {
7070
/** */
7171
public static final String LAST_EVT_SENT_TIME_DESC = "Timestamp of last applied event to destination cluster";
7272

73+
/** */
74+
public static final String DFLT_REGEXP = "";
75+
7376
/** Handle only primary entry flag. */
7477
private boolean onlyPrimary = DFLT_IS_ONLY_PRIMARY;
7578

@@ -80,10 +83,10 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx {
8083
private CdcRegexManager regexManager;
8184

8285
/** Include regex template for cache names. */
83-
private String includeTemplate;
86+
private String includeTemplate = DFLT_REGEXP;
8487

8588
/** Exclude regex template for cache names. */
86-
private String excludeTemplate;
89+
private String excludeTemplate = DFLT_REGEXP;
8790

8891
/** Cache IDs. */
8992
protected Set<Integer> cachesIds;
@@ -275,7 +278,7 @@ public AbstractIgniteCdcStreamer setCaches(Set<String> caches) {
275278
}
276279

277280
/**
278-
* Sets include regex pattern that participate in CDC.
281+
* Sets include regex pattern that participates in CDC.
279282
*
280283
* @param includeTemplate Include regex template
281284
* @return {@code this} for chaining.
@@ -287,7 +290,7 @@ public AbstractIgniteCdcStreamer setIncludeTemplate(String includeTemplate) {
287290
}
288291

289292
/**
290-
* Sets exclude regex pattern that participate in CDC.
293+
* Sets exclude regex pattern that participates in CDC.
291294
*
292295
* @param excludeTemplate Exclude regex template
293296
* @return {@code this} for chaining.

modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -129,20 +129,25 @@ private boolean matchAndSave(String cacheName) {
129129
* @return True if cache name matches include pattern and doesn't match exclude pattern.
130130
*/
131131
private boolean matchesFilters(String cacheName) {
132-
return includeFilter.matcher(cacheName).matches() && excludeFilter.matcher(cacheName).matches();
132+
return includeFilter.matcher(cacheName).matches() && !excludeFilter.matcher(cacheName).matches();
133133
}
134134

135135
/**
136136
* Compiles regex patterns from user templates.
137137
*
138138
* @param includeTemplate Include regex template.
139139
* @param excludeTemplate Exclude regex template.
140-
* @throws PatternSyntaxException If the template's syntax is invalid
140+
* @throws IgniteException If the template's syntax is invalid
141141
*/
142142
public void compileRegexp(String includeTemplate, String excludeTemplate) {
143-
includeFilter = Pattern.compile(includeTemplate);
143+
try {
144+
includeFilter = Pattern.compile(includeTemplate);
144145

145-
excludeFilter = Pattern.compile(excludeTemplate);
146+
excludeFilter = Pattern.compile(excludeTemplate);
147+
}
148+
catch (PatternSyntaxException e) {
149+
throw new IgniteException("Invalid cache regexp template.", e);
150+
}
146151
}
147152

148153
/**

modules/cdc-ext/src/main/java/org/apache/ignite/cdc/conflictresolve/CacheVersionConflictResolverPluginProvider.java

Lines changed: 24 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -18,11 +18,12 @@
1818
package org.apache.ignite.cdc.conflictresolve;
1919

2020
import java.io.Serializable;
21-
import java.util.HashSet;
2221
import java.util.Set;
2322
import java.util.UUID;
2423
import java.util.regex.Pattern;
24+
import java.util.regex.PatternSyntaxException;
2525

26+
import org.apache.ignite.IgniteException;
2627
import org.apache.ignite.IgniteLogger;
2728
import org.apache.ignite.cluster.ClusterNode;
2829
import org.apache.ignite.internal.IgniteEx;
@@ -44,6 +45,9 @@
4445
* @see CacheVersionConflictResolver
4546
*/
4647
public class CacheVersionConflictResolverPluginProvider<C extends PluginConfiguration> implements PluginProvider<C> {
48+
/** */
49+
public static final String DFLT_REGEXP = "";
50+
4751
/** Cluster id. */
4852
private byte clusterId;
4953

@@ -69,10 +73,10 @@ public class CacheVersionConflictResolverPluginProvider<C extends PluginConfigur
6973
private CacheVersionConflictResolver resolver;
7074

7175
/** Include regex templates for cache names. */
72-
private Set<String> includeTemplates = new HashSet<>();
76+
private String includeTemplate = DFLT_REGEXP;
7377

7478
/** Exclude regex templates for cache names. */
75-
private Set<String> excludeTemplates = new HashSet<>();
79+
private String excludeTemplate = DFLT_REGEXP;
7680

7781
/** Log. */
7882
private IgniteLogger log;
@@ -153,14 +157,14 @@ public void setConflictResolver(CacheVersionConflictResolver resolver) {
153157
this.resolver = resolver;
154158
}
155159

156-
/** @param includeTemplates Include regex templates */
157-
public void setIncludeTemplates(Set<String> includeTemplates) {
158-
this.includeTemplates = includeTemplates;
160+
/** @param includeTemplate Include regex template */
161+
public void setIncludeTemplate(String includeTemplate) {
162+
this.includeTemplate = includeTemplate;
159163
}
160164

161-
/** @param excludeTemplates Exclude regex templates */
162-
public void setExcludeTemplates(Set<String> excludeTemplates) {
163-
this.excludeTemplates = excludeTemplates;
165+
/** @param excludeTemplate Exclude regex template */
166+
public void setExcludeTemplate(String excludeTemplate) {
167+
this.excludeTemplate = excludeTemplate;
164168
}
165169

166170
/** {@inheritDoc} */
@@ -199,19 +203,22 @@ public void setExcludeTemplates(Set<String> excludeTemplates) {
199203
}
200204

201205
/**
202-
* Match cache name with regex patterns.
206+
* Matches cache name with compiled regex patterns.
203207
*
204208
* @param cacheName Cache name.
209+
* @return True if cache name matches include pattern and doesn't match exclude pattern.
210+
* @throws IgniteException If the template's syntax is invalid
205211
*/
206212
private boolean matchesFilters(String cacheName) {
207-
boolean matchesInclude = includeTemplates.stream()
208-
.map(Pattern::compile)
209-
.anyMatch(pattern -> pattern.matcher(cacheName).matches());
213+
try {
214+
boolean matchesInclude = Pattern.compile(includeTemplate).matcher(cacheName).matches();
210215

211-
boolean notMatchesExclude = excludeTemplates.stream()
212-
.map(Pattern::compile)
213-
.noneMatch(pattern -> pattern.matcher(cacheName).matches());
216+
boolean matchesExclude = Pattern.compile(excludeTemplate).matcher(cacheName).matches();
214217

215-
return matchesInclude && notMatchesExclude;
218+
return matchesInclude && !matchesExclude;
219+
}
220+
catch (PatternSyntaxException e) {
221+
throw new IgniteException("Invalid cache regexp template.", e);
222+
}
216223
}
217224
}

modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -116,6 +116,9 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumerEx {
116116
/** Count of metadata markers sent description. */
117117
public static final String MARKERS_SENT_CNT_DESC = "Count of metadata markers sent to Kafka";
118118

119+
/** */
120+
public static final String DFLT_REGEXP = "";
121+
119122
/** Default value for the flag that indicates whether entries only from primary nodes should be handled. */
120123
public static final boolean DFLT_IS_ONLY_PRIMARY = false;
121124

@@ -154,10 +157,10 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumerEx {
154157
private CdcRegexManager regexManager;
155158

156159
/** Include regex template for cache names. */
157-
private String includeTemplate;
160+
private String includeTemplate = DFLT_REGEXP;
158161

159162
/** Exclude regex template for cache names. */
160-
private String excludeTemplate;
163+
private String excludeTemplate = DFLT_REGEXP;
161164

162165
/** Max batch size. */
163166
private int maxBatchSz = DFLT_MAX_BATCH_SIZE;
@@ -463,7 +466,7 @@ public IgniteToKafkaCdcStreamer setCaches(Collection<String> caches) {
463466
}
464467

465468
/**
466-
* Sets include regex pattern that participate in CDC.
469+
* Sets include regex pattern that participates in CDC.
467470
*
468471
* @param includeTemplate Include regex template.
469472
* @return {@code this} for chaining.
@@ -475,7 +478,7 @@ public IgniteToKafkaCdcStreamer setIncludeTemplate(String includeTemplate) {
475478
}
476479

477480
/**
478-
* Sets exclude regex pattern that participate in CDC.
481+
* Sets exclude regex pattern that participates in CDC.
479482
*
480483
* @param excludeTemplate Exclude regex template.
481484
* @return {@code this} for chaining.

modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java

Lines changed: 11 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@
3434
import java.util.concurrent.atomic.AtomicLong;
3535
import java.util.function.Supplier;
3636
import java.util.regex.Pattern;
37+
import java.util.regex.PatternSyntaxException;
3738

3839
import org.apache.ignite.IgniteCheckedException;
3940
import org.apache.ignite.IgniteException;
@@ -304,14 +305,20 @@ private boolean matchesRegexTemplates(Integer key) {
304305
* Matches cache name with compiled regex patterns.
305306
*
306307
* @param cacheName Cache name.
307-
* @return True if cache name match include patterns and don't match exclude patterns.
308+
* @return True if cache name matches include pattern and doesn't match exclude pattern.
309+
* @throws IgniteException If the template's syntax is invalid
308310
*/
309311
private boolean matchesFilters(String cacheName) {
310-
boolean matchesInclude = Pattern.compile(includeTemplate).matcher(cacheName).matches();
312+
try {
313+
boolean matchesInclude = Pattern.compile(includeTemplate).matcher(cacheName).matches();
311314

312-
boolean notMatchesExclude = Pattern.compile(excludeTemplate).matcher(cacheName).matches();
315+
boolean matchesExclude = Pattern.compile(excludeTemplate).matcher(cacheName).matches();
313316

314-
return matchesInclude && notMatchesExclude;
317+
return matchesInclude && !matchesExclude;
318+
}
319+
catch (PatternSyntaxException e) {
320+
throw new IgniteException("Invalid cache regexp template.", e);
321+
}
315322
}
316323

317324
/**

modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@
1919

2020
import java.util.Collection;
2121
import java.util.Map;
22-
import java.util.Set;
2322

2423
import org.apache.ignite.cdc.CdcConfiguration;
2524
import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
@@ -182,28 +181,28 @@ public void setCaches(Collection<String> caches) {
182181
}
183182

184183
/**
185-
* @return Include regex templates
184+
* @return Include regex template.
186185
*/
187186
public String getIncludeTemplate() {
188187
return includeTemplate;
189188
}
190189

191190
/**
192-
* @param includeTemplate Include regex templates
191+
* @param includeTemplate Include regex template.
193192
*/
194193
public void setIncludeTemplate(String includeTemplate) {
195194
this.includeTemplate = includeTemplate;
196195
}
197196

198197
/**
199-
* @return Exclude regex templates
198+
* @return Exclude regex template
200199
*/
201200
public String getExcludeTemplate() {
202201
return excludeTemplate;
203202
}
204203

205204
/**
206-
* @param excludeTemplate Exclude regex templates
205+
* @param excludeTemplate Exclude regex template.
207206
*/
208207
public void setExcludeTemplate(String excludeTemplate) {
209208
this.excludeTemplate = excludeTemplate;

modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java

Lines changed: 2 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,6 @@
2525
import java.util.EnumSet;
2626
import java.util.HashSet;
2727
import java.util.List;
28-
import java.util.Set;
2928
import java.util.concurrent.ThreadLocalRandom;
3029
import java.util.concurrent.TimeUnit;
3130
import java.util.function.BiConsumer;
@@ -213,8 +212,8 @@ private enum WaitDataMode {
213212

214213
cfgPlugin1.setClusterId(clusterId);
215214
cfgPlugin1.setCaches(new HashSet<>(Arrays.asList(ACTIVE_PASSIVE_CACHE, ACTIVE_ACTIVE_CACHE)));
216-
cfgPlugin1.setIncludeTemplates(new HashSet<>(Arrays.asList(REGEX_INCLUDE_PATTERN)));
217-
cfgPlugin1.setExcludeTemplates(new HashSet<>(Arrays.asList(REGEX_EXCLUDE_PATTERN)));
215+
cfgPlugin1.setIncludeTemplate(REGEX_INCLUDE_PATTERN);
216+
cfgPlugin1.setExcludeTemplate(REGEX_EXCLUDE_PATTERN);
218217
cfgPlugin1.setConflictResolveField("reqId");
219218

220219
cfg.setPluginProviders(cfgPlugin1);
@@ -581,9 +580,6 @@ public void testWithExpiryPolicy() throws Exception {
581580
* Active/Active mode means changes made in both clusters. */
582581
@Test
583582
public void testActiveActiveReplicationWithRegexFilters() throws Exception {
584-
Set<String> includeTemplates = new HashSet<>(Arrays.asList(REGEX_INCLUDE_PATTERN));
585-
Set<String> excludeTemplates = new HashSet<>(Arrays.asList(REGEX_EXCLUDE_PATTERN));
586-
587583
createCache(srcCluster[0], ACTIVE_ACTIVE_CACHE);
588584
createCache(destCluster[0], ACTIVE_ACTIVE_CACHE);
589585

@@ -624,9 +620,6 @@ public void testActiveActiveReplicationWithRegexFilters() throws Exception {
624620
* Active/Passive mode means changes made only in one cluster. */
625621
@Test
626622
public void testActivePassiveReplicationWithRegexFilters() throws Exception {
627-
Set<String> includeTemplates = new HashSet<>(Arrays.asList(REGEX_INCLUDE_PATTERN));
628-
Set<String> excludeTemplates = new HashSet<>(Arrays.asList(REGEX_EXCLUDE_PATTERN));
629-
630623
//Start CDC with only 'active-active-cache' in 'caches' property of CDC config
631624
List<IgniteInternalFuture<?>> futs = startActivePassiveCdcWithFilters(ACTIVE_PASSIVE_CACHE, REGEX_INCLUDE_PATTERN,
632625
REGEX_EXCLUDE_PATTERN);

modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@
2020
import java.util.ArrayList;
2121
import java.util.Collections;
2222
import java.util.List;
23-
import java.util.Set;
2423
import java.util.function.Function;
2524
import org.apache.ignite.Ignition;
2625
import org.apache.ignite.cdc.thin.IgniteToIgniteClientCdcStreamer;
@@ -46,7 +45,7 @@
4645
public class CdcIgniteToIgniteReplicationTest extends AbstractReplicationTest {
4746
/** {@inheritDoc} */
4847
@Override protected List<IgniteInternalFuture<?>> startActivePassiveCdc(String cache) {
49-
return startActivePassiveCdcWithFilters(cache, null, null);
48+
return startActivePassiveCdcWithFilters(cache, "", "");
5049
}
5150

5251
/** {@inheritDoc} */
@@ -64,7 +63,7 @@ public class CdcIgniteToIgniteReplicationTest extends AbstractReplicationTest {
6463

6564
/** {@inheritDoc} */
6665
@Override protected List<IgniteInternalFuture<?>> startActiveActiveCdc() {
67-
return startActiveActiveCdcWithFilters(null, null);
66+
return startActiveActiveCdcWithFilters("", "");
6867
}
6968

7069
/** {@inheritDoc} */
@@ -101,8 +100,8 @@ public class CdcIgniteToIgniteReplicationTest extends AbstractReplicationTest {
101100
* @param destCfg Ignite destination cluster configuration.
102101
* @param dest Ignite destination cluster.
103102
* @param cache Cache name to stream to kafka.
104-
* @param includeTemplate Include regex templates for cache names.
105-
* @param excludeTemplate Exclude regex templates for cache names.
103+
* @param includeTemplate Include regex template for cache names.
104+
* @param excludeTemplate Exclude regex template for cache names.
106105
* @param threadName Thread to run CDC instance.
107106
* @return Future for Change Data Capture application.
108107
*/

0 commit comments

Comments
 (0)