From dd51041fa6475158d9a585ed9dc0616c7f821944 Mon Sep 17 00:00:00 2001 From: Andrei Nadyktov Date: Fri, 30 Aug 2024 14:37:53 +0300 Subject: [PATCH 01/14] IGNITE-22530 CDC: Add regex filters for cache names --- .../ignite/cdc/AbstractIgniteCdcStreamer.java | 161 +++++++++++- .../ignite/cdc/IgniteToIgniteCdcStreamer.java | 6 +- ...VersionConflictResolverPluginProvider.java | 38 ++- .../AbstractKafkaToIgniteCdcStreamer.java | 10 +- .../cdc/kafka/IgniteToKafkaCdcStreamer.java | 161 +++++++++++- .../cdc/kafka/KafkaToIgniteCdcStreamer.java | 5 + .../KafkaToIgniteCdcStreamerApplier.java | 59 ++++- ...KafkaToIgniteCdcStreamerConfiguration.java | 37 +++ .../kafka/KafkaToIgniteClientCdcStreamer.java | 5 + .../thin/IgniteToIgniteClientCdcStreamer.java | 6 +- .../ignite/cdc/AbstractReplicationTest.java | 116 +++++++++ .../cdc/CdcIgniteToIgniteReplicationTest.java | 31 ++- .../apache/ignite/cdc/RegexFiltersTest.java | 233 ++++++++++++++++++ .../kafka/CdcKafkaReplicationAppsTest.java | 5 + .../cdc/kafka/CdcKafkaReplicationTest.java | 74 ++++-- 15 files changed, 909 insertions(+), 38 deletions(-) create mode 100644 modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersTest.java diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java index f56a9954e..a2f830f28 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java @@ -17,9 +17,19 @@ package org.apache.ignite.cdc; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Set; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; import java.util.stream.Collectors; + import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -67,12 +77,30 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { /** */ public static final String LAST_EVT_SENT_TIME_DESC = "Timestamp of last applied event to destination cluster"; + /** File with saved names of caches added by cache masks. */ + private static final String SAVED_CACHES_FILE = "caches"; + + /** CDC directory path. */ + private Path cdcDir; + /** Handle only primary entry flag. */ private boolean onlyPrimary = DFLT_IS_ONLY_PRIMARY; /** Cache names. */ private Set caches; + /** Include regex templates for cache names. */ + private Set includeTemplates = new HashSet<>(); + + /** Compiled include regex patterns for cache names. */ + private Set includeFilters; + + /** Exclude regex templates for cache names. */ + private Set excludeTemplates = new HashSet<>(); + + /** Compiled exclude regex patterns for cache names. */ + private Set excludeFilters; + /** Cache IDs. */ protected Set cachesIds; @@ -99,14 +127,28 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { protected IgniteLogger log; /** {@inheritDoc} */ - @Override public void start(MetricRegistry reg) { + @Override public void start(MetricRegistry reg, Path cdcDir) { A.notEmpty(caches, "caches"); + this.cdcDir = cdcDir; + cachesIds = caches.stream() .mapToInt(CU::cacheId) .boxed() .collect(Collectors.toSet()); + prepareRegexFilters(); + + try { + loadCaches().stream() + .filter(this::matchesFilters) + .map(CU::cacheId) + .forEach(cachesIds::add); + } + catch (IOException e) { + throw new IgniteException(e); + } + MetricRegistryImpl mreg = (MetricRegistryImpl)reg; this.evtsCnt = mreg.longMetric(EVTS_SENT_CNT, EVTS_SENT_CNT_DESC); @@ -144,10 +186,101 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { /** {@inheritDoc} */ @Override public void onCacheChange(Iterator cacheEvents) { cacheEvents.forEachRemaining(e -> { - // Just skip. Handle of cache events not supported. + matchWithRegexTemplates(e.configuration().getName()); }); } + /** + * Finds match between cache name and user's regex templates. + * If match found, adds this cache's id to id's list and saves cache name to file. + * + * @param cacheName Cache name. + */ + private void matchWithRegexTemplates(String cacheName) { + int cacheId = CU.cacheId(cacheName); + + if (!cachesIds.contains(cacheId) && matchesFilters(cacheName)) { + cachesIds.add(cacheId); + + try { + saveCache(cacheName); + } + catch (IOException e) { + throw new IgniteException(e); + } + + if (log.isInfoEnabled()) + log.info("Cache has been added to replication [cacheName=" + cacheName + "]"); + } + } + + /** + * Writes cache name to file + * + * @param cacheName Cache name. + */ + private void saveCache(String cacheName) throws IOException { + if (cdcDir != null) { + Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); + + String cn = cacheName + '\n'; + + Files.write(savedCachesPath, cn.getBytes(), StandardOpenOption.APPEND); + } + } + + /** + * Loads saved caches from file. + * + * @return List of saved caches names. + */ + private List loadCaches() throws IOException { + if (cdcDir != null) { + Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); + + if (Files.notExists(savedCachesPath)) { + Files.createFile(savedCachesPath); + + if (log.isInfoEnabled()) + log.info("Cache list created: " + savedCachesPath); + } + + return Files.readAllLines(savedCachesPath); + } + return Collections.emptyList(); + } + + /** + * Compiles regex patterns from user templates. + * + * @throws PatternSyntaxException If the template's syntax is invalid + */ + private void prepareRegexFilters() { + includeFilters = includeTemplates.stream() + .map(Pattern::compile) + .collect(Collectors.toSet()); + + excludeFilters = excludeTemplates.stream() + .map(Pattern::compile) + .collect(Collectors.toSet()); + } + + /** + * Matches cache name with compiled regex patterns. + * + * @param cacheName Cache name. + * @return True if cache name match include patterns and don't match exclude patterns. + */ + private boolean matchesFilters(String cacheName) { + boolean matchesInclude = includeFilters.stream() + .anyMatch(pattern -> pattern.matcher(cacheName).matches()); + + boolean notMatchesExclude = excludeFilters.stream() + .noneMatch(pattern -> pattern.matcher(cacheName).matches()); + + return matchesInclude && notMatchesExclude; + } + /** {@inheritDoc} */ @Override public void onCacheDestroy(Iterator caches) { caches.forEachRemaining(e -> { @@ -238,6 +371,30 @@ public AbstractIgniteCdcStreamer setCaches(Set caches) { return this; } + /** + * Sets include regex patterns that participate in CDC. + * + * @param includeTemplates Include regex templates + * @return {@code this} for chaining. + */ + public AbstractIgniteCdcStreamer setIncludeTemplates(Set includeTemplates) { + this.includeTemplates = includeTemplates; + + return this; + } + + /** + * Sets exclude regex patterns that participate in CDC. + * + * @param excludeTemplates Exclude regex templates + * @return {@code this} for chaining. + */ + public AbstractIgniteCdcStreamer setExcludeTemplates(Set excludeTemplates) { + this.excludeTemplates = excludeTemplates; + + return this; + } + /** * Sets maximum batch size that will be applied to destination cluster. * diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/IgniteToIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/IgniteToIgniteCdcStreamer.java index d38f45883..597993719 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/IgniteToIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/IgniteToIgniteCdcStreamer.java @@ -17,6 +17,8 @@ package org.apache.ignite.cdc; +import java.nio.file.Path; + import org.apache.ignite.IgniteException; import org.apache.ignite.Ignition; import org.apache.ignite.cdc.conflictresolve.CacheVersionConflictResolverImpl; @@ -59,8 +61,8 @@ public class IgniteToIgniteCdcStreamer extends AbstractIgniteCdcStreamer impleme private volatile boolean alive = true; /** {@inheritDoc} */ - @Override public void start(MetricRegistry mreg) { - super.start(mreg); + @Override public void start(MetricRegistry mreg, Path cdcDir) { + super.start(mreg, cdcDir); if (log.isInfoEnabled()) log.info("Ignite To Ignite Streamer [cacheIds=" + cachesIds + ']'); diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/conflictresolve/CacheVersionConflictResolverPluginProvider.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/conflictresolve/CacheVersionConflictResolverPluginProvider.java index 0083f136a..f283b6c2c 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/conflictresolve/CacheVersionConflictResolverPluginProvider.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/conflictresolve/CacheVersionConflictResolverPluginProvider.java @@ -18,8 +18,11 @@ package org.apache.ignite.cdc.conflictresolve; import java.io.Serializable; +import java.util.HashSet; import java.util.Set; import java.util.UUID; +import java.util.regex.Pattern; + import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteEx; @@ -65,6 +68,12 @@ public class CacheVersionConflictResolverPluginProvider includeTemplates = new HashSet<>(); + + /** Exclude regex templates for cache names. */ + private Set excludeTemplates = new HashSet<>(); + /** Log. */ private IgniteLogger log; @@ -98,7 +107,7 @@ public CacheVersionConflictResolverPluginProvider() { @Override public CachePluginProvider createCacheProvider(CachePluginContext ctx) { String cacheName = ctx.igniteCacheConfiguration().getName(); - if (caches.contains(cacheName)) { + if (caches.contains(cacheName) || matchesFilters(cacheName)) { log.info("ConflictResolver provider set for cache [cacheName=" + cacheName + ']'); return provider; @@ -144,6 +153,16 @@ public void setConflictResolver(CacheVersionConflictResolver resolver) { this.resolver = resolver; } + /** @param includeTemplates Include regex templates */ + public void setIncludeTemplates(Set includeTemplates) { + this.includeTemplates = includeTemplates; + } + + /** @param excludeTemplates Exclude regex templates */ + public void setExcludeTemplates(Set excludeTemplates) { + this.excludeTemplates = excludeTemplates; + } + /** {@inheritDoc} */ @Override public void start(PluginContext ctx) { ((IgniteEx)ctx.grid()).context().cache().context().versions().dataCenterId(clusterId); @@ -178,4 +197,21 @@ public void setConflictResolver(CacheVersionConflictResolver resolver) { @Nullable @Override public T createComponent(PluginContext ctx, Class cls) { return null; } + + /** + * Match cache name with regex patterns. + * + * @param cacheName Cache name. + */ + private boolean matchesFilters(String cacheName) { + boolean matchesInclude = includeTemplates.stream() + .map(Pattern::compile) + .anyMatch(pattern -> pattern.matcher(cacheName).matches()); + + boolean notMatchesExclude = excludeTemplates.stream() + .map(Pattern::compile) + .noneMatch(pattern -> pattern.matcher(cacheName).matches()); + + return matchesInclude && notMatchesExclude; + } } diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/AbstractKafkaToIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/AbstractKafkaToIgniteCdcStreamer.java index 5332f1bd1..da01403d6 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/AbstractKafkaToIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/AbstractKafkaToIgniteCdcStreamer.java @@ -181,7 +181,8 @@ protected void runAppliers() { caches, metaUpdr, stopped, - metrics + metrics, + this ); addAndStart("applier-thread-" + cntr++, applier); @@ -252,6 +253,13 @@ private void addAndStart(String threadName, /** Checks that configured caches exist in a destination cluster. */ protected abstract void checkCaches(Collection caches); + /** + * Get cache names from client. + * + * @return Cache names. + * */ + protected abstract Collection getCaches(); + /** */ private void ackAsciiLogo(IgniteLogger log) { String ver = "ver. " + ACK_VER_STR; diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java index 08fa25a07..435c64d3a 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java @@ -17,8 +17,14 @@ package org.apache.ignite.cdc.kafka; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Properties; @@ -28,8 +34,12 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.Function; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; import java.util.stream.Collectors; import java.util.stream.IntStream; + +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryType; import org.apache.ignite.cdc.CdcCacheEvent; @@ -147,6 +157,24 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumer { /** Cache names. */ private Collection caches; + /** File with saved names of caches added by cache masks. */ + private static final String SAVED_CACHES_FILE = "caches"; + + /** CDC directory path. */ + private Path cdcDir; + + /** Include regex templates for cache names. */ + private Set includeTemplates = new HashSet<>(); + + /** Compiled include regex patterns for cache names. */ + private Set includeFilters; + + /** Exclude regex templates for cache names. */ + private Set excludeTemplates = new HashSet<>(); + + /** Compiled exclude regex patterns for cache names. */ + private Set excludeFilters; + /** Max batch size. */ private int maxBatchSz = DFLT_MAX_BATCH_SIZE; @@ -246,7 +274,7 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumer { /** {@inheritDoc} */ @Override public void onCacheChange(Iterator cacheEvents) { cacheEvents.forEachRemaining(e -> { - // Just skip. Handle of cache events not supported. + matchWithRegexTemplates(e.configuration().getName()); }); } @@ -318,7 +346,7 @@ private void sendOneBatch( } /** {@inheritDoc} */ - @Override public void start(MetricRegistry reg) { + @Override public void start(MetricRegistry reg, Path cdcDir) { A.notNull(kafkaProps, "Kafka properties"); A.notNull(evtTopic, "Kafka topic"); A.notNull(metadataTopic, "Kafka metadata topic"); @@ -329,10 +357,24 @@ private void sendOneBatch( kafkaProps.setProperty(KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class.getName()); kafkaProps.setProperty(VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + this.cdcDir = cdcDir; + cachesIds = caches.stream() .map(CU::cacheId) .collect(Collectors.toSet()); + prepareRegexFilters(); + + try { + loadCaches().stream() + .filter(this::matchesFilters) + .map(CU::cacheId) + .forEach(cachesIds::add); + } + catch (IOException e) { + throw new IgniteException(e); + } + try { producer = new KafkaProducer<>(kafkaProps); @@ -378,6 +420,97 @@ public IgniteToKafkaCdcStreamer setOnlyPrimary(boolean onlyPrimary) { return this; } + /** + * Compiles regex patterns from user templates. + * + * @throws PatternSyntaxException If the template's syntax is invalid + */ + private void prepareRegexFilters() { + includeFilters = includeTemplates.stream() + .map(Pattern::compile) + .collect(Collectors.toSet()); + + excludeFilters = excludeTemplates.stream() + .map(Pattern::compile) + .collect(Collectors.toSet()); + } + + /** + * Loads saved caches from file. + * + * @return List of saved caches names. + */ + private List loadCaches() throws IOException { + if (cdcDir != null) { + Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); + + if (Files.notExists(savedCachesPath)) { + Files.createFile(savedCachesPath); + + if (log.isInfoEnabled()) + log.info("Cache list created: " + savedCachesPath); + } + + return Files.readAllLines(savedCachesPath); + } + return Collections.emptyList(); + } + + /** + * Matches cache name with compiled regex patterns. + * + * @param cacheName Cache name. + * @return True if cache name match include patterns and don't match exclude patterns. + */ + private boolean matchesFilters(String cacheName) { + boolean matchesInclude = includeFilters.stream() + .anyMatch(pattern -> pattern.matcher(cacheName).matches()); + + boolean notMatchesExclude = excludeFilters.stream() + .noneMatch(pattern -> pattern.matcher(cacheName).matches()); + + return matchesInclude && notMatchesExclude; + } + + /** + * Finds match between cache name and user's regex templates. + * If match found, adds this cache's id to id's list and saves cache name to file. + * + * @param cacheName Cache name. + */ + private void matchWithRegexTemplates(String cacheName) { + int cacheId = CU.cacheId(cacheName); + + if (!cachesIds.contains(cacheId) && matchesFilters(cacheName)) { + cachesIds.add(cacheId); + + try { + saveCache(cacheName); + } + catch (IOException e) { + throw new IgniteException(e); + } + + if (log.isInfoEnabled()) + log.info("Cache has been added to replication [cacheName=" + cacheName + "]"); + } + } + + /** + * Writes cache name to file. + * + * @param cacheName Cache name. + */ + private void saveCache(String cacheName) throws IOException { + if (cdcDir != null) { + Path savedCaches = cdcDir.resolve(SAVED_CACHES_FILE); + + String cn = cacheName + '\n'; + + Files.write(savedCaches, cn.getBytes(), StandardOpenOption.APPEND); + } + } + /** * Sets topic that is used to send data to Kafka. * @@ -426,6 +559,30 @@ public IgniteToKafkaCdcStreamer setCaches(Collection caches) { return this; } + /** + * Sets include regex patterns that participate in CDC. + * + * @param includeTemplates Include regex templates. + * @return {@code this} for chaining. + */ + public IgniteToKafkaCdcStreamer setIncludeTemplates(Set includeTemplates) { + this.includeTemplates = includeTemplates; + + return this; + } + + /** + * Sets exclude regex patterns that participate in CDC. + * + * @param excludeTemplates Exclude regex templates + * @return {@code this} for chaining. + */ + public IgniteToKafkaCdcStreamer setExcludeTemplates(Set excludeTemplates) { + this.excludeTemplates = excludeTemplates; + + return this; + } + /** * Sets maximum batch size. * diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamer.java index 0e2ada723..6e9336747 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamer.java @@ -125,4 +125,9 @@ public KafkaToIgniteCdcStreamer( @Override protected void checkCaches(Collection caches) { caches.forEach(name -> Objects.requireNonNull(ign.cache(name), name + " not exists!")); } + + /** {@inheritDoc} */ + @Override protected Collection getCaches() { + return ign.cacheNames(); + } } diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java index c28e18ba9..4b67189c4 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java @@ -27,11 +27,14 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; +import java.util.regex.Pattern; + import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -42,6 +45,7 @@ import org.apache.ignite.internal.processors.cache.version.CacheVersionConflictResolver; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -104,6 +108,12 @@ class KafkaToIgniteCdcStreamerApplier implements Runnable, AutoCloseable { /** Caches ids to read. */ private final Set caches; + /** Include regex templates for cache names. */ + private final Set includeTemplates; + + /** Exclude regex templates for cache names. */ + private final Set excludeTemplates; + /** The maximum time to complete Kafka related requests, in milliseconds. */ private final long kafkaReqTimeout; @@ -128,6 +138,9 @@ class KafkaToIgniteCdcStreamerApplier implements Runnable, AutoCloseable { /** CDC kafka to ignite metrics */ private final KafkaToIgniteMetrics metrics; + /** Instance of KafkaToIgniteCdcStreamer */ + private final AbstractKafkaToIgniteCdcStreamer streamer; + /** * @param applierSupplier Cdc events applier supplier. * @param log Logger. @@ -139,6 +152,7 @@ class KafkaToIgniteCdcStreamerApplier implements Runnable, AutoCloseable { * @param metaUpdr Metadata updater. * @param stopped Stopped flag. * @param metrics CDC Kafka to Ignite metrics. + * @param streamer Instance of KafkaToIgniteCdcStreamer */ public KafkaToIgniteCdcStreamerApplier( Supplier applierSupplier, @@ -150,7 +164,8 @@ public KafkaToIgniteCdcStreamerApplier( Set caches, KafkaToIgniteMetadataUpdater metaUpdr, AtomicBoolean stopped, - KafkaToIgniteMetrics metrics + KafkaToIgniteMetrics metrics, + AbstractKafkaToIgniteCdcStreamer streamer ) { this.applierSupplier = applierSupplier; this.kafkaProps = kafkaProps; @@ -164,6 +179,9 @@ public KafkaToIgniteCdcStreamerApplier( this.stopped = stopped; this.log = log.getLogger(KafkaToIgniteCdcStreamerApplier.class); this.metrics = metrics; + this.streamer = streamer; + this.includeTemplates = streamerCfg.getIncludeTemplates(); + this.excludeTemplates = streamerCfg.getExcludeTemplates(); } /** {@inheritDoc} */ @@ -260,7 +278,44 @@ private boolean filterAndPossiblyUpdateMetadata(ConsumerRecord metrics.incrementReceivedEvents(); - return F.isEmpty(caches) || caches.contains(rec.key()); + return F.isEmpty(caches) || caches.contains(rec.key()) || matchesRegexTemplates(rec.key()); + } + + /** + * Gets caches names from CDC client and finds match + * between cache id and user's regex templates. + * + * @param key Cache id. + * @return True if match is found. + */ + private boolean matchesRegexTemplates(Integer key) { + Optional cache = streamer.getCaches().stream() + .filter(name -> CU.cacheId(name) == key) + .findAny(); + + Optional matchedCache = cache.filter(this::matchesFilters); + + matchedCache.ifPresent(c -> caches.add(CU.cacheId(c))); + + return matchedCache.isPresent(); + } + + /** + * Matches cache name with compiled regex patterns. + * + * @param cacheName Cache name. + * @return True if cache name match include patterns and don't match exclude patterns. + */ + private boolean matchesFilters(String cacheName) { + boolean matchesInclude = includeTemplates.stream() + .map(Pattern::compile) + .anyMatch(pattern -> pattern.matcher(cacheName).matches()); + + boolean notMatchesExclude = excludeTemplates.stream() + .map(Pattern::compile) + .noneMatch(pattern -> pattern.matcher(cacheName).matches()); + + return matchesInclude && notMatchesExclude; } /** diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java index 571afac8d..ece45402e 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java @@ -18,7 +18,10 @@ package org.apache.ignite.cdc.kafka; import java.util.Collection; +import java.util.HashSet; import java.util.Map; +import java.util.Set; + import org.apache.ignite.cdc.CdcConfiguration; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.spi.metric.MetricExporterSpi; @@ -83,6 +86,12 @@ public class KafkaToIgniteCdcStreamerConfiguration { */ private Collection caches; + /** Include regex templates for cache names. */ + private Set includeTemplates = new HashSet<>(); + + /** Exclude regex templates for cache names. */ + private Set excludeTemplates = new HashSet<>(); + /** Metric exporter SPI. */ private MetricExporterSpi[] metricExporterSpi; @@ -173,6 +182,34 @@ public void setCaches(Collection caches) { this.caches = caches; } + /** + * @return Include regex templates + */ + public Set getIncludeTemplates() { + return includeTemplates; + } + + /** + * @param includeTemplates Include regex templates + */ + public void setIncludeTemplates(Set includeTemplates) { + this.includeTemplates = includeTemplates; + } + + /** + * @return Exclude regex templates + */ + public Set getExcludeTemplates() { + return excludeTemplates; + } + + /** + * @param excludeTemplates Exclude regex templates + */ + public void setExcludeTemplates(Set excludeTemplates) { + this.excludeTemplates = excludeTemplates; + } + /** @return The maximum time to complete Kafka related requests, in milliseconds. */ public long getKafkaRequestTimeout() { return kafkaReqTimeout; diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteClientCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteClientCdcStreamer.java index 706d53ad0..14ef7d229 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteClientCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteClientCdcStreamer.java @@ -125,4 +125,9 @@ public KafkaToIgniteClientCdcStreamer( caches.forEach(name -> A.ensure(clusterCaches.contains(name), name + " not exists!")); } + + /** {@inheritDoc} */ + @Override protected Collection getCaches() { + return client.cacheNames(); + } } diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/thin/IgniteToIgniteClientCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/thin/IgniteToIgniteClientCdcStreamer.java index bc7af745b..607ca19c6 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/thin/IgniteToIgniteClientCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/thin/IgniteToIgniteClientCdcStreamer.java @@ -17,6 +17,8 @@ package org.apache.ignite.cdc.thin; +import java.nio.file.Path; + import org.apache.ignite.Ignition; import org.apache.ignite.cdc.AbstractIgniteCdcStreamer; import org.apache.ignite.cdc.conflictresolve.CacheVersionConflictResolverImpl; @@ -66,8 +68,8 @@ public class IgniteToIgniteClientCdcStreamer extends AbstractIgniteCdcStreamer { private long aliveCheckTimeout = DFLT_ALIVE_CHECK_TIMEOUT; /** {@inheritDoc} */ - @Override public void start(MetricRegistry mreg) { - super.start(mreg); + @Override public void start(MetricRegistry mreg, Path cdcDir) { + super.start(mreg, cdcDir); if (log.isInfoEnabled()) log.info("Ignite To Ignite Client Streamer [cacheIds=" + cachesIds + ']'); diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java index 9a5e18c7f..c79657f76 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java @@ -25,6 +25,7 @@ import java.util.EnumSet; import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; @@ -146,6 +147,18 @@ public static Collection parameters() { /** */ public static final String IGNORED_CACHE = "ignored-cache"; + /** */ + public static final String REGEX_INCLUDE_TEMPLATE_CACHE = "cdc_on_cache"; + + /** */ + public static final String REGEX_EXCLUDE_TEMPLATE_CACHE = "cdc_on_excluded_cache"; + + /** */ + public static final String REGEX_INCLUDE_PATTERN = "cdc_on.*"; + + /** */ + public static final String REGEX_EXCLUDE_PATTERN = "cdc_on_excluded.*"; + /** */ public static final byte SRC_CLUSTER_ID = 1; @@ -200,6 +213,8 @@ private enum WaitDataMode { cfgPlugin1.setClusterId(clusterId); cfgPlugin1.setCaches(new HashSet<>(Arrays.asList(ACTIVE_PASSIVE_CACHE, ACTIVE_ACTIVE_CACHE))); + cfgPlugin1.setIncludeTemplates(new HashSet<>(Arrays.asList(REGEX_INCLUDE_PATTERN))); + cfgPlugin1.setExcludeTemplates(new HashSet<>(Arrays.asList(REGEX_EXCLUDE_PATTERN))); cfgPlugin1.setConflictResolveField("reqId"); cfg.setPluginProviders(cfgPlugin1); @@ -562,6 +577,98 @@ public void testWithExpiryPolicy() throws Exception { } } + /** Check that caches matching regex filters in config, are added to CDC after its creation. + * Active/Active mode means changes made in both clusters. */ + @Test + public void testActiveActiveReplicationWithRegexFilters() throws Exception { + Set includeTemplates = new HashSet<>(Arrays.asList(REGEX_INCLUDE_PATTERN)); + Set excludeTemplates = new HashSet<>(Arrays.asList(REGEX_EXCLUDE_PATTERN)); + + createCache(srcCluster[0], ACTIVE_ACTIVE_CACHE); + createCache(destCluster[0], ACTIVE_ACTIVE_CACHE); + + IgniteCache srcCache = createCache(srcCluster[0], REGEX_INCLUDE_TEMPLATE_CACHE); + IgniteCache destCache = createCache(destCluster[0], REGEX_INCLUDE_TEMPLATE_CACHE); + + // Even keys goes to src cluster. + runAsync(generateData(REGEX_INCLUDE_TEMPLATE_CACHE, srcCluster[srcCluster.length - 1], + IntStream.range(0, KEYS_CNT).filter(i -> i % 2 == 0))); + + // Odd keys goes to dest cluster. + runAsync(generateData(REGEX_INCLUDE_TEMPLATE_CACHE, destCluster[destCluster.length - 1], + IntStream.range(0, KEYS_CNT).filter(i -> i % 2 != 0))); + + //Start CDC with only 'active-active-cache' in 'caches' property of CDC config + List> futs = startActiveActiveCdcWithFilters(includeTemplates, excludeTemplates); + + try { + waitForSameData(srcCache, destCache, KEYS_CNT, WaitDataMode.EXISTS, futs); + + runAsync(() -> IntStream.range(0, KEYS_CNT).filter(j -> j % 2 == 0).forEach(srcCache::remove)); + runAsync(() -> IntStream.range(0, KEYS_CNT).filter(j -> j % 2 != 0).forEach(destCache::remove)); + + waitForSameData(srcCache, destCache, KEYS_CNT, WaitDataMode.REMOVED, futs); + + //Shouldn't add to the replication, otherwise CDC will throw an error + runAsync(generateData(REGEX_EXCLUDE_TEMPLATE_CACHE, srcCluster[srcCluster.length - 1], IntStream.range(0, KEYS_CNT))); + + assertFalse(destCluster[0].cacheNames().contains(REGEX_EXCLUDE_TEMPLATE_CACHE)); + } + finally { + for (IgniteInternalFuture fut : futs) + fut.cancel(); + } + } + + /** Check that caches matching regex filters in config, are added to CDC after its creation. + * Active/Passive mode means changes made only in one cluster. */ + @Test + public void testActivePassiveReplicationWithRegexFilters() throws Exception { + Set includeTemplates = new HashSet<>(Arrays.asList(REGEX_INCLUDE_PATTERN)); + Set excludeTemplates = new HashSet<>(Arrays.asList(REGEX_EXCLUDE_PATTERN)); + + //Start CDC with only 'active-active-cache' in 'caches' property of CDC config + List> futs = startActivePassiveCdcWithFilters(ACTIVE_PASSIVE_CACHE, + includeTemplates, excludeTemplates); + + try { + createCache(destCluster[0], ACTIVE_PASSIVE_CACHE); + + IgniteCache destCache = createCache(destCluster[0], REGEX_INCLUDE_TEMPLATE_CACHE); + + // Updates for "ignored-cache" should be ignored because of CDC consume configuration. + runAsync(generateData(IGNORED_CACHE, srcCluster[srcCluster.length - 1], IntStream.range(0, KEYS_CNT))); + runAsync(generateData(REGEX_INCLUDE_TEMPLATE_CACHE, srcCluster[srcCluster.length - 1], IntStream.range(0, KEYS_CNT))); + + IgniteCache srcCache = + createCache(srcCluster[srcCluster.length - 1], REGEX_INCLUDE_TEMPLATE_CACHE); + + waitForSameData(srcCache, destCache, KEYS_CNT, WaitDataMode.EXISTS, futs); + + checkMetricsCount(KEYS_CNT); + checkMetrics(); + + IntStream.range(0, KEYS_CNT).forEach(srcCache::remove); + + waitForSameData(srcCache, destCache, KEYS_CNT, WaitDataMode.REMOVED, futs); + + checkMetrics(); + + assertFalse(destCluster[0].cacheNames().contains(IGNORED_CACHE)); + + checkMetricsCount(2 * KEYS_CNT); + + //Shouldn't add to the replication, otherwise CDC will throw an error + runAsync(generateData(REGEX_EXCLUDE_TEMPLATE_CACHE, srcCluster[srcCluster.length - 1], IntStream.range(0, KEYS_CNT))); + + assertFalse(destCluster[0].cacheNames().contains(REGEX_EXCLUDE_TEMPLATE_CACHE)); + } + finally { + for (IgniteInternalFuture fut : futs) + fut.cancel(); + } + } + /** */ public Runnable generateData(String cacheName, IgniteEx ign, IntStream keys) { return () -> { @@ -688,9 +795,18 @@ protected String[] hostAddresses(IgniteEx[] dest) { /** */ protected abstract List> startActivePassiveCdc(String cache); + /** */ + protected abstract List> startActivePassiveCdcWithFilters(String cache, + Set includeTemplates, + Set excludeTemplates); + /** */ protected abstract List> startActiveActiveCdc(); + /** */ + protected abstract List> startActiveActiveCdcWithFilters(Set includeTemplates, + Set excludeTemplates); + /** */ protected abstract void checkConsumerMetrics(Function longMetric); diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java index b6d42e240..9c021645c 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Set; import java.util.function.Function; import org.apache.ignite.Ignition; import org.apache.ignite.cdc.thin.IgniteToIgniteClientCdcStreamer; @@ -45,26 +46,40 @@ public class CdcIgniteToIgniteReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActivePassiveCdc(String cache) { + return startActivePassiveCdcWithFilters(cache, Collections.emptySet(), Collections.emptySet()); + } + + /** {@inheritDoc} */ + @Override protected List> startActivePassiveCdcWithFilters(String cache, + Set includeTemplates, + Set excludeTemplates) { List> futs = new ArrayList<>(); for (int i = 0; i < srcCluster.length; i++) - futs.add(igniteToIgnite(srcCluster[i].configuration(), destClusterCliCfg[i], destCluster, cache, "ignite-to-ignite-src-" + i)); + futs.add(igniteToIgnite(srcCluster[i].configuration(), destClusterCliCfg[i], destCluster, cache, + includeTemplates, excludeTemplates, "ignite-to-ignite-src-" + i)); return futs; } /** {@inheritDoc} */ @Override protected List> startActiveActiveCdc() { + return startActiveActiveCdcWithFilters(Collections.emptySet(), Collections.emptySet()); + } + + /** {@inheritDoc} */ + @Override protected List> startActiveActiveCdcWithFilters(Set includeTemplates, + Set excludeTemplates) { List> futs = new ArrayList<>(); for (int i = 0; i < srcCluster.length; i++) { - futs.add(igniteToIgnite( - srcCluster[i].configuration(), destClusterCliCfg[i], destCluster, ACTIVE_ACTIVE_CACHE, "ignite-to-ignite-src-" + i)); + futs.add(igniteToIgnite(srcCluster[i].configuration(), destClusterCliCfg[i], destCluster, + ACTIVE_ACTIVE_CACHE, includeTemplates, excludeTemplates, "ignite-to-ignite-src-" + i)); } for (int i = 0; i < destCluster.length; i++) { - futs.add(igniteToIgnite( - destCluster[i].configuration(), srcClusterCliCfg[i], srcCluster, ACTIVE_ACTIVE_CACHE, "ignite-to-ignite-dest-" + i)); + futs.add(igniteToIgnite(destCluster[i].configuration(), srcClusterCliCfg[i], srcCluster, + ACTIVE_ACTIVE_CACHE, includeTemplates, excludeTemplates, "ignite-to-ignite-dest-" + i)); } return futs; @@ -86,6 +101,8 @@ public class CdcIgniteToIgniteReplicationTest extends AbstractReplicationTest { * @param destCfg Ignite destination cluster configuration. * @param dest Ignite destination cluster. * @param cache Cache name to stream to kafka. + * @param includeTemplates Include regex templates for cache names. + * @param excludeTemplates Exclude regex templates for cache names. * @param threadName Thread to run CDC instance. * @return Future for Change Data Capture application. */ @@ -94,6 +111,8 @@ protected IgniteInternalFuture igniteToIgnite( IgniteConfiguration destCfg, IgniteEx[] dest, String cache, + Set includeTemplates, + Set excludeTemplates, @Nullable String threadName ) { return runAsync(() -> { @@ -115,6 +134,8 @@ protected IgniteInternalFuture igniteToIgnite( streamer.setMaxBatchSize(KEYS_CNT); streamer.setCaches(Collections.singleton(cache)); + streamer.setIncludeTemplates(includeTemplates); + streamer.setExcludeTemplates(excludeTemplates); cdcCfg.setConsumer(streamer); cdcCfg.setMetricExporterSpi(new JmxMetricExporterSpi()); diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersTest.java new file mode 100644 index 000000000..e4007ab08 --- /dev/null +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersTest.java @@ -0,0 +1,233 @@ +package org.apache.ignite.cdc; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.stream.IntStream; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cdc.thin.IgniteToIgniteClientCdcStreamer; +import org.apache.ignite.cluster.ClusterState; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.ClientConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.cdc.CdcMain; +import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.spi.metric.jmx.JmxMetricExporterSpi; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; + +import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.DFLT_PORT_RANGE; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; + +/** */ +public class RegexFiltersTest extends GridCommonAbstractTest { + + /** */ + private IgniteEx src; + + /** */ + private IgniteEx dest; + + /** */ + private int discoPort = TcpDiscoverySpi.DFLT_PORT; + + /** */ + private enum WaitDataMode { + /** */ + EXISTS, + + /** */ + REMOVED + } + + /** */ + private static final String TEST_CACHE = "test-cache"; + + /** */ + private static final String REGEX_MATCHING_CACHE = "regex-cache"; + + /** */ + private static final String REGEX_INCLUDE_PATTERN = "regex.*"; + + /** */ + private Set includeTemplates; + + /** */ + private static final int KEYS_CNT = 1000; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + TcpDiscoveryVmIpFinder finder = new TcpDiscoveryVmIpFinder() + .setAddresses(Collections.singleton("127.0.0.1:" + discoPort + ".." + (discoPort + DFLT_PORT_RANGE))); + + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName) + .setDiscoverySpi(new TcpDiscoverySpi() + .setLocalPort(discoPort) + .setIpFinder(finder)); + + cfg.setDataStorageConfiguration(new DataStorageConfiguration() + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setPersistenceEnabled(true) + .setCdcEnabled(true))); + + cfg.getDataStorageConfiguration() + .setWalForceArchiveTimeout(5_000); + + cfg.setConsistentId(igniteInstanceName); + + return cfg; + } + + /** + * + * @param srcCfg Ignite source node configuration. + * @param cache Cache name to stream to Ignite2Ignite. + * @param includeTemplates Include cache templates. + * @param excludeTemplates Exclude cache templates. + * @return Future for Change Data Capture application. + */ + private IgniteInternalFuture startCdc(IgniteConfiguration srcCfg, + String cache, + Set includeTemplates, + Set excludeTemplates) { + return runAsync(() -> { + CdcConfiguration cdcCfg = new CdcConfiguration(); + + AbstractIgniteCdcStreamer streamer = new IgniteToIgniteClientCdcStreamer() + .setDestinationClientConfiguration(new ClientConfiguration() + .setAddresses(F.first(dest.localNode().addresses()) + ":" + + dest.localNode().attribute(ClientListenerProcessor.CLIENT_LISTENER_PORT))); + + streamer.setMaxBatchSize(KEYS_CNT); + streamer.setCaches(Collections.singleton(cache)); + streamer.setIncludeTemplates(includeTemplates); + streamer.setExcludeTemplates(excludeTemplates); + + cdcCfg.setConsumer(streamer); + cdcCfg.setMetricExporterSpi(new JmxMetricExporterSpi()); + + CdcMain cdc = new CdcMain(srcCfg, null, cdcCfg); + + cdc.run(); + }); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + cleanPersistenceDir(); + + src = startGrid(getConfiguration("source-cluster")); + + discoPort += DFLT_PORT_RANGE + 1; + + dest = startGrid(getConfiguration("dest-cluster")); + + includeTemplates = new HashSet<>(Arrays.asList(REGEX_INCLUDE_PATTERN)); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** */ + public void waitForSameData( + IgniteCache src, + IgniteCache dest, + int keysCnt, + WaitDataMode mode, + IgniteInternalFuture fut + ) throws IgniteInterruptedCheckedException { + assertTrue(waitForCondition(() -> { + for (int i = 0; i < keysCnt; i++) { + if (mode == WaitDataMode.EXISTS) { + if (!src.containsKey(i) || !dest.containsKey(i)) + return checkFut(false, fut); + } + else if (mode == WaitDataMode.REMOVED) { + if (src.containsKey(i) || dest.containsKey(i)) + return checkFut(false, fut); + + continue; + } + else + throw new IllegalArgumentException(mode + " not supported."); + + Integer data = dest.get(i); + + if (!data.equals(src.get(i))) + return checkFut(false, fut); + } + + return checkFut(true, fut); + }, getTestTimeout())); + } + + /** */ + private boolean checkFut(boolean res, IgniteInternalFuture fut) { + assertFalse("Fut error: " + X.getFullStackTrace(fut.error()), fut.isDone()); + + return res; + } + + /** */ + public Runnable generateData(IgniteCache cache, IntStream keys) { + return () -> { + keys.forEach(i -> cache.put(i, i * 2)); + }; + } + + /** + * Test checks whether caches added by regex filters are saved to and read from file after CDC restart. + */ + @Test + public void testRegexFiltersOnCdcRestart() throws Exception { + + src.cluster().state(ClusterState.ACTIVE); + + dest.cluster().state(ClusterState.ACTIVE); + + //Start CDC only with 'test-cache' in config and cache masks (regex filters) + IgniteInternalFuture cdc = startCdc(src.configuration(), TEST_CACHE, includeTemplates, Collections.emptySet()); + + IgniteCache srcCache = src.getOrCreateCache(new CacheConfiguration() + .setName(REGEX_MATCHING_CACHE) + .setCacheMode(CacheMode.PARTITIONED) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)); + + IgniteCache destCache = dest.getOrCreateCache(new CacheConfiguration() + .setName(REGEX_MATCHING_CACHE) + .setCacheMode(CacheMode.PARTITIONED) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)); + + cdc.cancel(); + + //Restart CDC + IgniteInternalFuture cdc2 = startCdc(src.configuration(), TEST_CACHE, includeTemplates, Collections.emptySet()); + + try { + runAsync(generateData(srcCache, IntStream.range(0, KEYS_CNT))); + + waitForSameData(srcCache, destCache, KEYS_CNT, WaitDataMode.EXISTS, cdc2); + } + finally { + cdc2.cancel(); + } + } +} diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java index ef4e14004..e2f20df1a 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java @@ -25,6 +25,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; @@ -113,6 +114,8 @@ public class CdcKafkaReplicationAppsTest extends CdcKafkaReplicationTest { String topic, String metadataTopic, String cache, + Set includeTemplates, + Set excludeTemplates, String threadName ) { Map params = new HashMap<>(); @@ -141,6 +144,8 @@ public class CdcKafkaReplicationAppsTest extends CdcKafkaReplicationTest { IgniteEx[] dest, int partFrom, int partTo, + Set includeTemplates, + Set excludeTemplates, String threadName ) { Map params = new HashMap<>(); diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java index a56b2941b..773f1cec0 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.List; import java.util.Properties; +import java.util.Set; import java.util.function.Function; import java.util.function.Supplier; import javax.management.DynamicMBean; @@ -93,6 +94,13 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActivePassiveCdc(String cache) { + return startActivePassiveCdcWithFilters(cache, Collections.emptySet(), Collections.emptySet()); + } + + /** {@inheritDoc} */ + @Override protected List> startActivePassiveCdcWithFilters(String cache, + Set includeTemplates, + Set excludeTemplates) { try { KAFKA.createTopic(cache, DFLT_PARTS, 1); @@ -107,7 +115,7 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { for (IgniteEx ex : srcCluster) { int idx = getTestIgniteInstanceIndex(ex.name()); - futs.add(igniteToKafka(ex.configuration(), cache, SRC_DEST_META_TOPIC, cache, "ignite-src-to-kafka-" + idx)); + futs.add(igniteToKafka(ex.configuration(), cache, SRC_DEST_META_TOPIC, cache, includeTemplates, excludeTemplates, "ignite-src-to-kafka-" + idx)); } for (int i = 0; i < destCluster.length; i++) { @@ -119,7 +127,9 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { destCluster, i * (DFLT_PARTS / 2), (i + 1) * (DFLT_PARTS / 2), - "kafka-to-ignite-dest-" + i + includeTemplates, + excludeTemplates, + "kafka-to-ignite-dest-" + i )); } @@ -128,20 +138,26 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActiveActiveCdc() { + return startActiveActiveCdcWithFilters(Collections.emptySet(), Collections.emptySet()); + } + + /** {@inheritDoc} */ + @Override protected List> startActiveActiveCdcWithFilters(Set includeTemplates, + Set excludeTemplates) { List> futs = new ArrayList<>(); for (IgniteEx ex : srcCluster) { int idx = getTestIgniteInstanceIndex(ex.name()); - - futs.add(igniteToKafka( - ex.configuration(), SRC_DEST_TOPIC, SRC_DEST_META_TOPIC, ACTIVE_ACTIVE_CACHE, "ignite-src-to-kafka-" + idx)); + + futs.add(igniteToKafka(ex.configuration(), SRC_DEST_TOPIC, SRC_DEST_META_TOPIC, ACTIVE_ACTIVE_CACHE, includeTemplates, + excludeTemplates, "ignite-src-to-kafka-" + idx)); } for (IgniteEx ex : destCluster) { int idx = getTestIgniteInstanceIndex(ex.name()); - - futs.add(igniteToKafka( - ex.configuration(), DEST_SRC_TOPIC, DEST_SRC_META_TOPIC, ACTIVE_ACTIVE_CACHE, "ignite-dest-to-kafka-" + idx)); + + futs.add(igniteToKafka(ex.configuration(), DEST_SRC_TOPIC, DEST_SRC_META_TOPIC, ACTIVE_ACTIVE_CACHE, includeTemplates, + excludeTemplates, "ignite-dest-to-kafka-" + idx)); } futs.add(kafkaToIgnite( @@ -152,6 +168,8 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { destCluster, 0, DFLT_PARTS, + includeTemplates, + excludeTemplates, "kafka-to-ignite-src" )); @@ -163,6 +181,8 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { srcCluster, 0, DFLT_PARTS, + includeTemplates, + excludeTemplates, "kafka-to-ignite-dest" )); @@ -255,25 +275,31 @@ private void checkK2IMetrics(Function longMetric) { * @param topic Kafka topic name. * @param metadataTopic Metadata topic name. * @param cache Cache name to stream to kafka. + * @param includeTemplates Include regex templates for cache names. + * @param excludeTemplates Exclude regex templates for cache names. * @return Future for Change Data Capture application. */ protected IgniteInternalFuture igniteToKafka( - IgniteConfiguration igniteCfg, - String topic, - String metadataTopic, - String cache, - String threadName + IgniteConfiguration igniteCfg, + String topic, + String metadataTopic, + String cache, + Set includeTemplates, + Set excludeTemplates, + String threadName ) { return runAsync(() -> { IgniteToKafkaCdcStreamer cdcCnsmr = new IgniteToKafkaCdcStreamer() - .setTopic(topic) - .setMetadataTopic(metadataTopic) - .setKafkaPartitions(DFLT_PARTS) - .setCaches(Collections.singleton(cache)) - .setMaxBatchSize(KEYS_CNT) - .setOnlyPrimary(false) - .setKafkaProperties(kafkaProperties()) - .setKafkaRequestTimeout(DFLT_KAFKA_REQ_TIMEOUT); + .setTopic(topic) + .setMetadataTopic(metadataTopic) + .setKafkaPartitions(DFLT_PARTS) + .setCaches(Collections.singleton(cache)) + .setIncludeTemplates(includeTemplates) + .setExcludeTemplates(excludeTemplates) + .setMaxBatchSize(KEYS_CNT) + .setOnlyPrimary(false) + .setKafkaProperties(kafkaProperties()) + .setKafkaRequestTimeout(DFLT_KAFKA_REQ_TIMEOUT); CdcConfiguration cdcCfg = new CdcConfiguration(); @@ -292,6 +318,8 @@ protected IgniteInternalFuture igniteToKafka( * @param cacheName Cache name. * @param igniteCfg Ignite configuration. * @param dest Destination Ignite cluster. + * @param includeTemplates Include regex templates for cache names. + * @param excludeTemplates Exclude regex templates for cache names. * @return Future for runed {@link KafkaToIgniteCdcStreamer}. */ protected IgniteInternalFuture kafkaToIgnite( @@ -302,6 +330,8 @@ protected IgniteInternalFuture kafkaToIgnite( IgniteEx[] dest, int fromPart, int toPart, + Set includeTemplates, + Set excludeTemplates, String threadName ) { KafkaToIgniteCdcStreamerConfiguration cfg = new KafkaToIgniteCdcStreamerConfiguration(); @@ -311,6 +341,8 @@ protected IgniteInternalFuture kafkaToIgnite( cfg.setThreadCount((toPart - fromPart) / 2); cfg.setCaches(Collections.singletonList(cacheName)); + cfg.setIncludeTemplates(includeTemplates); + cfg.setExcludeTemplates(excludeTemplates); cfg.setTopic(topic); cfg.setMetadataTopic(metadataTopic); cfg.setKafkaRequestTimeout(DFLT_KAFKA_REQ_TIMEOUT); From 7967fac2262d3c6e7bc4038fb4113b9f05aca017 Mon Sep 17 00:00:00 2001 From: Andrei Nadyktov Date: Fri, 11 Oct 2024 16:28:09 +0300 Subject: [PATCH 02/14] IGNITE-22530 Make caches set in KafkaToIgniteCdcStreamerApplier mutable --- .../cdc/kafka/AbstractKafkaToIgniteCdcStreamer.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/AbstractKafkaToIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/AbstractKafkaToIgniteCdcStreamer.java index da01403d6..40102a50c 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/AbstractKafkaToIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/AbstractKafkaToIgniteCdcStreamer.java @@ -19,11 +19,11 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cdc.AbstractCdcEventsApplier; @@ -152,13 +152,13 @@ public AbstractKafkaToIgniteCdcStreamer(Properties kafkaProps, KafkaToIgniteCdcS protected void runAppliers() { AtomicBoolean stopped = new AtomicBoolean(); - Set caches = null; + Set caches = new HashSet<>(); if (!F.isEmpty(streamerCfg.getCaches())) { checkCaches(streamerCfg.getCaches()); - caches = streamerCfg.getCaches().stream() - .map(CU::cacheId).collect(Collectors.toSet()); + streamerCfg.getCaches().stream() + .map(CU::cacheId).forEach(caches::add); } KafkaToIgniteMetadataUpdater metaUpdr = new KafkaToIgniteMetadataUpdater( From e654826044c458c0baf92121fd193ec5b91a9ee5 Mon Sep 17 00:00:00 2001 From: Andrei Nadyktov Date: Sat, 12 Oct 2024 22:18:13 +0300 Subject: [PATCH 03/14] IGNITE-22530 Add removal of destroyed caches from cacheList file --- .../ignite/cdc/AbstractIgniteCdcStreamer.java | 44 +++++++++++++++++-- .../cdc/kafka/IgniteToKafkaCdcStreamer.java | 44 +++++++++++++++++-- 2 files changed, 82 insertions(+), 6 deletions(-) diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java index a2f830f28..1fd6cba2b 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java @@ -25,6 +25,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; @@ -283,9 +284,46 @@ private boolean matchesFilters(String cacheName) { /** {@inheritDoc} */ @Override public void onCacheDestroy(Iterator caches) { - caches.forEachRemaining(e -> { - // Just skip. Handle of cache events not supported. - }); + caches.forEachRemaining(this::deleteRegexpCacheIfPresent); + } + + /** + * Removes cache added by regexp from cache list, if this cache is present in file, to prevent file size overflow. + * + * @param cacheId Cache id. + */ + private void deleteRegexpCacheIfPresent(Integer cacheId) { + try { + List caches = loadCaches(); + + Optional cacheName = caches.stream() + .filter(name -> CU.cacheId(name) == cacheId) + .findAny(); + + if (cacheName.isPresent()) { + String name = cacheName.get(); + + caches.remove(name); + + Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); + + StringBuilder cacheList = new StringBuilder(); + + for (String cache : caches) { + cacheList.append(cache); + + cacheList.append('\n'); + } + + Files.write(savedCachesPath, cacheList.toString().getBytes()); + + if (log.isInfoEnabled()) + log.info("Cache has been removed from replication [cacheName=" + name + ']'); + } + } + catch (IOException e) { + throw new IgniteException(e); + } } /** {@inheritDoc} */ diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java index 435c64d3a..5cfaaadb3 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java @@ -27,6 +27,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.Properties; import java.util.Set; import java.util.concurrent.ExecutionException; @@ -280,9 +281,46 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumer { /** {@inheritDoc} */ @Override public void onCacheDestroy(Iterator caches) { - caches.forEachRemaining(e -> { - // Just skip. Handle of cache events not supported. - }); + caches.forEachRemaining(this::deleteRegexpCacheIfPresent); + } + + /** + * Removes cache added by regexp from cache list, if this cache is present in file, to prevent file size overflow. + * + * @param cacheId Cache id. + */ + private void deleteRegexpCacheIfPresent(Integer cacheId) { + try { + List caches = loadCaches(); + + Optional cacheName = caches.stream() + .filter(name -> CU.cacheId(name) == cacheId) + .findAny(); + + if (cacheName.isPresent()) { + String name = cacheName.get(); + + caches.remove(name); + + Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); + + StringBuilder cacheList = new StringBuilder(); + + for (String cache : caches) { + cacheList.append(cache); + + cacheList.append('\n'); + } + + Files.write(savedCachesPath, cacheList.toString().getBytes()); + + if (log.isInfoEnabled()) + log.info("Cache has been removed from replication [cacheName=" + name + ']'); + } + } + catch (IOException e) { + throw new IgniteException(e); + } } /** Send marker(meta need to be updated) record to each partition of events topic. */ From 8c51e9dd479433161d81cb820fb0d8043f214982 Mon Sep 17 00:00:00 2001 From: Andrey Nadyktov Date: Thu, 14 Nov 2024 01:25:23 +0300 Subject: [PATCH 04/14] IGNITE-22530 Add atomic write to caches file --- .../ignite/cdc/AbstractIgniteCdcStreamer.java | 71 ++++++++++--------- .../cdc/kafka/IgniteToKafkaCdcStreamer.java | 71 ++++++++++--------- .../cdc/kafka/CdcKafkaReplicationTest.java | 11 +-- 3 files changed, 84 insertions(+), 69 deletions(-) diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java index 1fd6cba2b..0553dc683 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java @@ -20,8 +20,6 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.StandardOpenOption; -import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -46,6 +44,8 @@ import org.apache.ignite.metric.MetricRegistry; import org.apache.ignite.resources.LoggerResource; +import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; +import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; import static org.apache.ignite.cdc.kafka.IgniteToKafkaCdcStreamer.DFLT_IS_ONLY_PRIMARY; /** @@ -81,6 +81,9 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { /** File with saved names of caches added by cache masks. */ private static final String SAVED_CACHES_FILE = "caches"; + /** Temporary file with saved names of caches added by cache masks. */ + private static final String SAVED_CACHES_TMP_FILE = "caches_tmp"; + /** CDC directory path. */ private Path cdcDir; @@ -193,7 +196,7 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { /** * Finds match between cache name and user's regex templates. - * If match found, adds this cache's id to id's list and saves cache name to file. + * If match is found, adds this cache's id to id's list and saves cache name to file. * * @param cacheName Cache name. */ @@ -204,7 +207,11 @@ private void matchWithRegexTemplates(String cacheName) { cachesIds.add(cacheId); try { - saveCache(cacheName); + List caches = loadCaches(); + + caches.add(cacheName); + + save(caches); } catch (IOException e) { throw new IgniteException(e); @@ -216,18 +223,28 @@ private void matchWithRegexTemplates(String cacheName) { } /** - * Writes cache name to file + * Writes caches list to file * - * @param cacheName Cache name. + * @param caches Caches list. */ - private void saveCache(String cacheName) throws IOException { - if (cdcDir != null) { - Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); + private void save(List caches) throws IOException { + if (cdcDir == null) { + throw new IgniteException("Can't write to '" + SAVED_CACHES_FILE + "' file. Cdc directory is null"); + } + Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); + Path tmpSavedCachesPath = cdcDir.resolve(SAVED_CACHES_TMP_FILE); - String cn = cacheName + '\n'; + StringBuilder cacheList = new StringBuilder(); - Files.write(savedCachesPath, cn.getBytes(), StandardOpenOption.APPEND); + for (String cache : caches) { + cacheList.append(cache); + + cacheList.append('\n'); } + + Files.write(tmpSavedCachesPath, cacheList.toString().getBytes()); + + Files.move(tmpSavedCachesPath, savedCachesPath, ATOMIC_MOVE, REPLACE_EXISTING); } /** @@ -236,19 +253,19 @@ private void saveCache(String cacheName) throws IOException { * @return List of saved caches names. */ private List loadCaches() throws IOException { - if (cdcDir != null) { - Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); - - if (Files.notExists(savedCachesPath)) { - Files.createFile(savedCachesPath); + if (cdcDir == null) { + throw new IgniteException("Can't load '" + SAVED_CACHES_FILE + "' file. Cdc directory is null"); + } + Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); - if (log.isInfoEnabled()) - log.info("Cache list created: " + savedCachesPath); - } + if (Files.notExists(savedCachesPath)) { + Files.createFile(savedCachesPath); - return Files.readAllLines(savedCachesPath); + if (log.isInfoEnabled()) + log.info("Cache list created: " + savedCachesPath); } - return Collections.emptyList(); + + return Files.readAllLines(savedCachesPath); } /** @@ -305,17 +322,7 @@ private void deleteRegexpCacheIfPresent(Integer cacheId) { caches.remove(name); - Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); - - StringBuilder cacheList = new StringBuilder(); - - for (String cache : caches) { - cacheList.append(cache); - - cacheList.append('\n'); - } - - Files.write(savedCachesPath, cacheList.toString().getBytes()); + save(caches); if (log.isInfoEnabled()) log.info("Cache has been removed from replication [cacheName=" + name + ']'); diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java index 5cfaaadb3..769c77e65 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java @@ -20,10 +20,8 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.StandardOpenOption; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -65,6 +63,8 @@ import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.IntegerSerializer; +import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; +import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; import static org.apache.ignite.cdc.kafka.KafkaToIgniteCdcStreamerConfiguration.DFLT_KAFKA_REQ_TIMEOUT; import static org.apache.ignite.cdc.kafka.KafkaToIgniteCdcStreamerConfiguration.DFLT_MAX_BATCH_SIZE; import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; @@ -161,6 +161,9 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumer { /** File with saved names of caches added by cache masks. */ private static final String SAVED_CACHES_FILE = "caches"; + /** Temporary file with saved names of caches added by cache masks. */ + private static final String SAVED_CACHES_TMP_FILE = "caches_tmp"; + /** CDC directory path. */ private Path cdcDir; @@ -302,17 +305,7 @@ private void deleteRegexpCacheIfPresent(Integer cacheId) { caches.remove(name); - Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); - - StringBuilder cacheList = new StringBuilder(); - - for (String cache : caches) { - cacheList.append(cache); - - cacheList.append('\n'); - } - - Files.write(savedCachesPath, cacheList.toString().getBytes()); + save(caches); if (log.isInfoEnabled()) log.info("Cache has been removed from replication [cacheName=" + name + ']'); @@ -479,19 +472,19 @@ private void prepareRegexFilters() { * @return List of saved caches names. */ private List loadCaches() throws IOException { - if (cdcDir != null) { - Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); - - if (Files.notExists(savedCachesPath)) { - Files.createFile(savedCachesPath); + if (cdcDir == null) { + throw new IgniteException("Can't load '" + SAVED_CACHES_FILE + "' file. Cdc directory is null"); + } + Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); - if (log.isInfoEnabled()) - log.info("Cache list created: " + savedCachesPath); - } + if (Files.notExists(savedCachesPath)) { + Files.createFile(savedCachesPath); - return Files.readAllLines(savedCachesPath); + if (log.isInfoEnabled()) + log.info("Cache list created: " + savedCachesPath); } - return Collections.emptyList(); + + return Files.readAllLines(savedCachesPath); } /** @@ -512,7 +505,7 @@ private boolean matchesFilters(String cacheName) { /** * Finds match between cache name and user's regex templates. - * If match found, adds this cache's id to id's list and saves cache name to file. + * If match is found, adds this cache's id to id's list and saves cache name to file. * * @param cacheName Cache name. */ @@ -523,7 +516,11 @@ private void matchWithRegexTemplates(String cacheName) { cachesIds.add(cacheId); try { - saveCache(cacheName); + List caches = loadCaches(); + + caches.add(cacheName); + + save(caches); } catch (IOException e) { throw new IgniteException(e); @@ -535,18 +532,28 @@ private void matchWithRegexTemplates(String cacheName) { } /** - * Writes cache name to file. + * Writes caches list to file * - * @param cacheName Cache name. + * @param caches Caches list. */ - private void saveCache(String cacheName) throws IOException { - if (cdcDir != null) { - Path savedCaches = cdcDir.resolve(SAVED_CACHES_FILE); + private void save(List caches) throws IOException { + if (cdcDir == null) { + throw new IgniteException("Can't write to '" + SAVED_CACHES_FILE + "' file. Cdc directory is null"); + } + Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); + Path tmpSavedCachesPath = cdcDir.resolve(SAVED_CACHES_TMP_FILE); + + StringBuilder cacheList = new StringBuilder(); - String cn = cacheName + '\n'; + for (String cache : caches) { + cacheList.append(cache); - Files.write(savedCaches, cn.getBytes(), StandardOpenOption.APPEND); + cacheList.append('\n'); } + + Files.write(tmpSavedCachesPath, cacheList.toString().getBytes()); + + Files.move(tmpSavedCachesPath, savedCachesPath, ATOMIC_MOVE, REPLACE_EXISTING); } /** diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java index 773f1cec0..72e75ce81 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java @@ -115,7 +115,8 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { for (IgniteEx ex : srcCluster) { int idx = getTestIgniteInstanceIndex(ex.name()); - futs.add(igniteToKafka(ex.configuration(), cache, SRC_DEST_META_TOPIC, cache, includeTemplates, excludeTemplates, "ignite-src-to-kafka-" + idx)); + futs.add(igniteToKafka(ex.configuration(), cache, SRC_DEST_META_TOPIC, cache, includeTemplates, + excludeTemplates, "ignite-src-to-kafka-" + idx)); } for (int i = 0; i < destCluster.length; i++) { @@ -149,15 +150,15 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { for (IgniteEx ex : srcCluster) { int idx = getTestIgniteInstanceIndex(ex.name()); - futs.add(igniteToKafka(ex.configuration(), SRC_DEST_TOPIC, SRC_DEST_META_TOPIC, ACTIVE_ACTIVE_CACHE, includeTemplates, - excludeTemplates, "ignite-src-to-kafka-" + idx)); + futs.add(igniteToKafka(ex.configuration(), SRC_DEST_TOPIC, SRC_DEST_META_TOPIC, ACTIVE_ACTIVE_CACHE, + includeTemplates, excludeTemplates, "ignite-src-to-kafka-" + idx)); } for (IgniteEx ex : destCluster) { int idx = getTestIgniteInstanceIndex(ex.name()); - futs.add(igniteToKafka(ex.configuration(), DEST_SRC_TOPIC, DEST_SRC_META_TOPIC, ACTIVE_ACTIVE_CACHE, includeTemplates, - excludeTemplates, "ignite-dest-to-kafka-" + idx)); + futs.add(igniteToKafka(ex.configuration(), DEST_SRC_TOPIC, DEST_SRC_META_TOPIC, ACTIVE_ACTIVE_CACHE, + includeTemplates, excludeTemplates, "ignite-dest-to-kafka-" + idx)); } futs.add(kafkaToIgnite( From e6a5e40571aba0341beff5068a2a5403abb6ad49 Mon Sep 17 00:00:00 2001 From: Andrei Nadyktov Date: Wed, 4 Jun 2025 18:36:29 +0300 Subject: [PATCH 05/14] IGNITE-22530 Add CdcConsumerEx interface --- .../org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java | 7 ++++++- .../ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java | 9 +++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java index 0553dc683..1df509366 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java @@ -53,7 +53,7 @@ * * @see AbstractCdcEventsApplier */ -public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { +public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx { /** */ public static final String EVTS_SENT_CNT = "EventsCount"; @@ -130,6 +130,11 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { @LoggerResource protected IgniteLogger log; + /** {@inheritDoc} */ + @Override public void start(MetricRegistry reg) { + //No-op + } + /** {@inheritDoc} */ @Override public void start(MetricRegistry reg, Path cdcDir) { A.notEmpty(caches, "caches"); diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java index 769c77e65..f3be71e02 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java @@ -42,7 +42,7 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryType; import org.apache.ignite.cdc.CdcCacheEvent; -import org.apache.ignite.cdc.CdcConsumer; +import org.apache.ignite.cdc.CdcConsumerEx; import org.apache.ignite.cdc.CdcEvent; import org.apache.ignite.cdc.TypeMapping; import org.apache.ignite.cdc.conflictresolve.CacheVersionConflictResolverImpl; @@ -87,7 +87,7 @@ * @see KafkaToIgniteClientCdcStreamer * @see CacheVersionConflictResolverImpl */ -public class IgniteToKafkaCdcStreamer implements CdcConsumer { +public class IgniteToKafkaCdcStreamer implements CdcConsumerEx { /** */ public static final String EVTS_SENT_CNT = "EventsCount"; @@ -376,6 +376,11 @@ private void sendOneBatch( } } + /** {@inheritDoc} */ + @Override public void start(MetricRegistry reg) { + //No-op + } + /** {@inheritDoc} */ @Override public void start(MetricRegistry reg, Path cdcDir) { A.notNull(kafkaProps, "Kafka properties"); From fd11e4f4073836c86272cdc49749e3382ab1bfb1 Mon Sep 17 00:00:00 2001 From: Andrei Nadyktov Date: Sun, 8 Jun 2025 22:05:40 +0300 Subject: [PATCH 06/14] IGNITE-22530 Add CdcRegexManager --- .../apache/ignite/cdc/CdcRegexManager.java | 223 ++++++++++++++++++ 1 file changed, 223 insertions(+) create mode 100644 modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java new file mode 100644 index 000000000..5d3cf7cf5 --- /dev/null +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java @@ -0,0 +1,223 @@ +/* + * 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.ignite.cdc; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; +import java.util.stream.Collectors; + +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.util.typedef.internal.CU; + +import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; +import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; + +/** + * TODO: Add JavaDoc + */ +public class CdcRegexManager implements CdcRegexMatcher { + + /** File with saved names of caches added by cache masks. */ + private static final String SAVED_CACHES_FILE = "caches"; + + /** Temporary file with saved names of caches added by cache masks. */ + private static final String SAVED_CACHES_TMP_FILE = "caches_tmp"; + + /** CDC directory path. */ + private final Path cdcDir; + + /** Include regex patterns for cache names. */ + private Set includeFilters; + + /** Exclude regex patterns for cache names. */ + private Set excludeFilters; + + /** Logger. */ + private IgniteLogger log; + + public CdcRegexManager(Path cdcDir, IgniteLogger log) { + this.cdcDir = cdcDir; + this.log = log; + } + + /** {@inheritDoc} */ + @Override public boolean match(String cacheName) { + return matchAndSave(cacheName); + } + + /** + * Get actual list of names of caches added by regex templates from cache list file. + * Caches that added to replication through regex templates during the work of CDC application, + * are saved to file so they can be restored after application restart. + * + * @return Caches names list. + */ + public List getSavedCaches() { + try { + return loadCaches().stream() + .filter(this::matchesFilters) + .collect(Collectors.toList()); + } + catch (IOException e) { + throw new IgniteException(e); + } + } + + /** + * Finds match between cache name and user's regex templates. + * If match is found, saves cache name to file. + * + * @param cacheName Cache name. + * @return True if cache name matches user's regexp patterns. + */ + private boolean matchAndSave(String cacheName) { + if (matchesFilters(cacheName)) { + try { + List caches = loadCaches(); + + caches.add(cacheName); + + save(caches); + } + catch (IOException e) { + throw new IgniteException(e); + } + + if (log.isInfoEnabled()) + log.info("Cache has been added to replication [cacheName=" + cacheName + "]"); + + return true; + } + return false; + } + + /** + * Matches cache name with compiled regex patterns. + * + * @param cacheName Cache name. + * @return True if cache name matches include patterns and doesn't match exclude patterns. + */ + private boolean matchesFilters(String cacheName) { + boolean matchesInclude = includeFilters.stream() + .anyMatch(pattern -> pattern.matcher(cacheName).matches()); + + boolean notMatchesExclude = excludeFilters.stream() + .noneMatch(pattern -> pattern.matcher(cacheName).matches()); + + return matchesInclude && notMatchesExclude; + } + + /** + * Compiles regex patterns from user templates. + * + * @param includeTemplates Include regex templates. + * @param excludeTemplates Exclude regex templates. + * @throws PatternSyntaxException If the template's syntax is invalid + */ + public void compileRegexp(Set includeTemplates, Set excludeTemplates) { + includeFilters = includeTemplates.stream() + .map(Pattern::compile) + .collect(Collectors.toSet()); + + excludeFilters = excludeTemplates.stream() + .map(Pattern::compile) + .collect(Collectors.toSet()); + } + + /** + * Loads saved CDC caches from file. If file not found, creates a new one containing empty list. + * + * @return List of saved caches names. + */ + private List loadCaches() throws IOException { + if (cdcDir == null) { + throw new IgniteException("Can't load '" + SAVED_CACHES_FILE + "' file. Cdc directory is null"); + } + Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); + + if (Files.notExists(savedCachesPath)) { + Files.createFile(savedCachesPath); + + if (log.isInfoEnabled()) + log.info("Cache list created: " + savedCachesPath); + } + + return Files.readAllLines(savedCachesPath); + } + + /** + * Writes caches list to file. + * + * @param caches Caches list. + */ + private void save(List caches) throws IOException { + if (cdcDir == null) { + throw new IgniteException("Can't write to '" + SAVED_CACHES_FILE + "' file. Cdc directory is null"); + } + Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); + Path tmpSavedCachesPath = cdcDir.resolve(SAVED_CACHES_TMP_FILE); + + StringBuilder cacheList = new StringBuilder(); + + for (String cache : caches) { + cacheList.append(cache); + + cacheList.append('\n'); + } + + Files.write(tmpSavedCachesPath, cacheList.toString().getBytes()); + + Files.move(tmpSavedCachesPath, savedCachesPath, ATOMIC_MOVE, REPLACE_EXISTING); + } + + /** + * Removes cache added by regexp from cache list if such cache is present in file to prevent disk space overflow. + * + * @param cacheId Cache id. + */ + public void deleteRegexpCacheIfPresent(Integer cacheId) { + try { + List caches = loadCaches(); + + Optional cacheName = caches.stream() + .filter(name -> CU.cacheId(name) == cacheId) + .findAny(); + + if (cacheName.isPresent()) { + String name = cacheName.get(); + + caches.remove(name); + + save(caches); + + if (log.isInfoEnabled()) + log.info("Cache has been removed from replication [cacheName=" + name + ']'); + } + } + catch (IOException e) { + throw new IgniteException(e); + } + } +} From b137e512edb3a20fcf647a5392e47f53b820f16b Mon Sep 17 00:00:00 2001 From: Andrei Nadyktov Date: Sun, 8 Jun 2025 22:38:47 +0300 Subject: [PATCH 07/14] IGNITE-22530 Refactor AbstractIgniteCdcStreamer for use with CdcRegexManager --- .../ignite/cdc/AbstractIgniteCdcStreamer.java | 173 ++---------------- 1 file changed, 13 insertions(+), 160 deletions(-) diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java index 1df509366..c3867e241 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java @@ -17,16 +17,10 @@ package org.apache.ignite.cdc; -import java.io.IOException; -import java.nio.file.Files; import java.nio.file.Path; import java.util.HashSet; import java.util.Iterator; -import java.util.List; -import java.util.Optional; import java.util.Set; -import java.util.regex.Pattern; -import java.util.regex.PatternSyntaxException; import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; @@ -44,8 +38,6 @@ import org.apache.ignite.metric.MetricRegistry; import org.apache.ignite.resources.LoggerResource; -import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; -import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; import static org.apache.ignite.cdc.kafka.IgniteToKafkaCdcStreamer.DFLT_IS_ONLY_PRIMARY; /** @@ -78,33 +70,21 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx { /** */ public static final String LAST_EVT_SENT_TIME_DESC = "Timestamp of last applied event to destination cluster"; - /** File with saved names of caches added by cache masks. */ - private static final String SAVED_CACHES_FILE = "caches"; - - /** Temporary file with saved names of caches added by cache masks. */ - private static final String SAVED_CACHES_TMP_FILE = "caches_tmp"; - - /** CDC directory path. */ - private Path cdcDir; - /** Handle only primary entry flag. */ private boolean onlyPrimary = DFLT_IS_ONLY_PRIMARY; /** Cache names. */ private Set caches; + /** Regexp manager. */ + private CdcRegexManager regexManager; + /** Include regex templates for cache names. */ private Set includeTemplates = new HashSet<>(); - /** Compiled include regex patterns for cache names. */ - private Set includeFilters; - /** Exclude regex templates for cache names. */ private Set excludeTemplates = new HashSet<>(); - /** Compiled exclude regex patterns for cache names. */ - private Set excludeFilters; - /** Cache IDs. */ protected Set cachesIds; @@ -139,24 +119,18 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx { @Override public void start(MetricRegistry reg, Path cdcDir) { A.notEmpty(caches, "caches"); - this.cdcDir = cdcDir; + regexManager = new CdcRegexManager(cdcDir, log); cachesIds = caches.stream() .mapToInt(CU::cacheId) .boxed() .collect(Collectors.toSet()); - prepareRegexFilters(); + regexManager.compileRegexp(includeTemplates, excludeTemplates); - try { - loadCaches().stream() - .filter(this::matchesFilters) - .map(CU::cacheId) - .forEach(cachesIds::add); - } - catch (IOException e) { - throw new IgniteException(e); - } + regexManager.getSavedCaches().stream() + .map(CU::cacheId) + .forEach(cachesIds::add); MetricRegistryImpl mreg = (MetricRegistryImpl)reg; @@ -195,147 +169,26 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx { /** {@inheritDoc} */ @Override public void onCacheChange(Iterator cacheEvents) { cacheEvents.forEachRemaining(e -> { - matchWithRegexTemplates(e.configuration().getName()); + matchWithRegex(e.configuration().getName()); }); } /** * Finds match between cache name and user's regex templates. - * If match is found, adds this cache's id to id's list and saves cache name to file. + * If match is found, adds this cache's id to id's list. * * @param cacheName Cache name. */ - private void matchWithRegexTemplates(String cacheName) { + private void matchWithRegex(String cacheName) { int cacheId = CU.cacheId(cacheName); - if (!cachesIds.contains(cacheId) && matchesFilters(cacheName)) { + if (!cachesIds.contains(cacheId) && regexManager.match(cacheName)) cachesIds.add(cacheId); - - try { - List caches = loadCaches(); - - caches.add(cacheName); - - save(caches); - } - catch (IOException e) { - throw new IgniteException(e); - } - - if (log.isInfoEnabled()) - log.info("Cache has been added to replication [cacheName=" + cacheName + "]"); - } - } - - /** - * Writes caches list to file - * - * @param caches Caches list. - */ - private void save(List caches) throws IOException { - if (cdcDir == null) { - throw new IgniteException("Can't write to '" + SAVED_CACHES_FILE + "' file. Cdc directory is null"); - } - Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); - Path tmpSavedCachesPath = cdcDir.resolve(SAVED_CACHES_TMP_FILE); - - StringBuilder cacheList = new StringBuilder(); - - for (String cache : caches) { - cacheList.append(cache); - - cacheList.append('\n'); - } - - Files.write(tmpSavedCachesPath, cacheList.toString().getBytes()); - - Files.move(tmpSavedCachesPath, savedCachesPath, ATOMIC_MOVE, REPLACE_EXISTING); - } - - /** - * Loads saved caches from file. - * - * @return List of saved caches names. - */ - private List loadCaches() throws IOException { - if (cdcDir == null) { - throw new IgniteException("Can't load '" + SAVED_CACHES_FILE + "' file. Cdc directory is null"); - } - Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); - - if (Files.notExists(savedCachesPath)) { - Files.createFile(savedCachesPath); - - if (log.isInfoEnabled()) - log.info("Cache list created: " + savedCachesPath); - } - - return Files.readAllLines(savedCachesPath); - } - - /** - * Compiles regex patterns from user templates. - * - * @throws PatternSyntaxException If the template's syntax is invalid - */ - private void prepareRegexFilters() { - includeFilters = includeTemplates.stream() - .map(Pattern::compile) - .collect(Collectors.toSet()); - - excludeFilters = excludeTemplates.stream() - .map(Pattern::compile) - .collect(Collectors.toSet()); - } - - /** - * Matches cache name with compiled regex patterns. - * - * @param cacheName Cache name. - * @return True if cache name match include patterns and don't match exclude patterns. - */ - private boolean matchesFilters(String cacheName) { - boolean matchesInclude = includeFilters.stream() - .anyMatch(pattern -> pattern.matcher(cacheName).matches()); - - boolean notMatchesExclude = excludeFilters.stream() - .noneMatch(pattern -> pattern.matcher(cacheName).matches()); - - return matchesInclude && notMatchesExclude; } /** {@inheritDoc} */ @Override public void onCacheDestroy(Iterator caches) { - caches.forEachRemaining(this::deleteRegexpCacheIfPresent); - } - - /** - * Removes cache added by regexp from cache list, if this cache is present in file, to prevent file size overflow. - * - * @param cacheId Cache id. - */ - private void deleteRegexpCacheIfPresent(Integer cacheId) { - try { - List caches = loadCaches(); - - Optional cacheName = caches.stream() - .filter(name -> CU.cacheId(name) == cacheId) - .findAny(); - - if (cacheName.isPresent()) { - String name = cacheName.get(); - - caches.remove(name); - - save(caches); - - if (log.isInfoEnabled()) - log.info("Cache has been removed from replication [cacheName=" + name + ']'); - } - } - catch (IOException e) { - throw new IgniteException(e); - } + caches.forEachRemaining(regexManager::deleteRegexpCacheIfPresent); } /** {@inheritDoc} */ From 6254a701a851bf83f035af9973ac1d0ffb446fd6 Mon Sep 17 00:00:00 2001 From: Andrey Nadyktov Date: Mon, 9 Jun 2025 21:58:53 +0300 Subject: [PATCH 08/14] IGNITE-22530 Refactor IgniteToKafkaCdcStreamer for use with CdcRegexManager --- .../cdc/kafka/IgniteToKafkaCdcStreamer.java | 170 ++---------------- 1 file changed, 12 insertions(+), 158 deletions(-) diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java index f3be71e02..6845a56cf 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java @@ -17,15 +17,12 @@ package org.apache.ignite.cdc.kafka; -import java.io.IOException; -import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Optional; import java.util.Properties; import java.util.Set; import java.util.concurrent.ExecutionException; @@ -33,17 +30,15 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.Function; -import java.util.regex.Pattern; -import java.util.regex.PatternSyntaxException; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryType; import org.apache.ignite.cdc.CdcCacheEvent; import org.apache.ignite.cdc.CdcConsumerEx; import org.apache.ignite.cdc.CdcEvent; +import org.apache.ignite.cdc.CdcRegexManager; import org.apache.ignite.cdc.TypeMapping; import org.apache.ignite.cdc.conflictresolve.CacheVersionConflictResolverImpl; import org.apache.ignite.internal.binary.BinaryTypeImpl; @@ -63,8 +58,6 @@ import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.IntegerSerializer; -import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; -import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; import static org.apache.ignite.cdc.kafka.KafkaToIgniteCdcStreamerConfiguration.DFLT_KAFKA_REQ_TIMEOUT; import static org.apache.ignite.cdc.kafka.KafkaToIgniteCdcStreamerConfiguration.DFLT_MAX_BATCH_SIZE; import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; @@ -158,27 +151,15 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumerEx { /** Cache names. */ private Collection caches; - /** File with saved names of caches added by cache masks. */ - private static final String SAVED_CACHES_FILE = "caches"; - - /** Temporary file with saved names of caches added by cache masks. */ - private static final String SAVED_CACHES_TMP_FILE = "caches_tmp"; - - /** CDC directory path. */ - private Path cdcDir; + /** Regexp manager. */ + private CdcRegexManager regexManager; /** Include regex templates for cache names. */ private Set includeTemplates = new HashSet<>(); - /** Compiled include regex patterns for cache names. */ - private Set includeFilters; - /** Exclude regex templates for cache names. */ private Set excludeTemplates = new HashSet<>(); - /** Compiled exclude regex patterns for cache names. */ - private Set excludeFilters; - /** Max batch size. */ private int maxBatchSz = DFLT_MAX_BATCH_SIZE; @@ -278,42 +259,13 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumerEx { /** {@inheritDoc} */ @Override public void onCacheChange(Iterator cacheEvents) { cacheEvents.forEachRemaining(e -> { - matchWithRegexTemplates(e.configuration().getName()); + matchWithRegex(e.configuration().getName()); }); } /** {@inheritDoc} */ @Override public void onCacheDestroy(Iterator caches) { - caches.forEachRemaining(this::deleteRegexpCacheIfPresent); - } - - /** - * Removes cache added by regexp from cache list, if this cache is present in file, to prevent file size overflow. - * - * @param cacheId Cache id. - */ - private void deleteRegexpCacheIfPresent(Integer cacheId) { - try { - List caches = loadCaches(); - - Optional cacheName = caches.stream() - .filter(name -> CU.cacheId(name) == cacheId) - .findAny(); - - if (cacheName.isPresent()) { - String name = cacheName.get(); - - caches.remove(name); - - save(caches); - - if (log.isInfoEnabled()) - log.info("Cache has been removed from replication [cacheName=" + name + ']'); - } - } - catch (IOException e) { - throw new IgniteException(e); - } + caches.forEachRemaining(regexManager::deleteRegexpCacheIfPresent); } /** Send marker(meta need to be updated) record to each partition of events topic. */ @@ -393,23 +345,17 @@ private void sendOneBatch( kafkaProps.setProperty(KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class.getName()); kafkaProps.setProperty(VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); - this.cdcDir = cdcDir; + regexManager = new CdcRegexManager(cdcDir, log); cachesIds = caches.stream() .map(CU::cacheId) .collect(Collectors.toSet()); - prepareRegexFilters(); + regexManager.compileRegexp(includeTemplates, excludeTemplates); - try { - loadCaches().stream() - .filter(this::matchesFilters) + regexManager.getSavedCaches().stream() .map(CU::cacheId) .forEach(cachesIds::add); - } - catch (IOException e) { - throw new IgniteException(e); - } try { producer = new KafkaProducer<>(kafkaProps); @@ -457,108 +403,16 @@ public IgniteToKafkaCdcStreamer setOnlyPrimary(boolean onlyPrimary) { } /** - * Compiles regex patterns from user templates. - * - * @throws PatternSyntaxException If the template's syntax is invalid - */ - private void prepareRegexFilters() { - includeFilters = includeTemplates.stream() - .map(Pattern::compile) - .collect(Collectors.toSet()); - - excludeFilters = excludeTemplates.stream() - .map(Pattern::compile) - .collect(Collectors.toSet()); - } - - /** - * Loads saved caches from file. - * - * @return List of saved caches names. - */ - private List loadCaches() throws IOException { - if (cdcDir == null) { - throw new IgniteException("Can't load '" + SAVED_CACHES_FILE + "' file. Cdc directory is null"); - } - Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); - - if (Files.notExists(savedCachesPath)) { - Files.createFile(savedCachesPath); - - if (log.isInfoEnabled()) - log.info("Cache list created: " + savedCachesPath); - } - - return Files.readAllLines(savedCachesPath); - } - - /** - * Matches cache name with compiled regex patterns. + * Finds a match between the cache name and user regex templates. + * If a match is found, adds the cache to replication. * * @param cacheName Cache name. - * @return True if cache name match include patterns and don't match exclude patterns. */ - private boolean matchesFilters(String cacheName) { - boolean matchesInclude = includeFilters.stream() - .anyMatch(pattern -> pattern.matcher(cacheName).matches()); - - boolean notMatchesExclude = excludeFilters.stream() - .noneMatch(pattern -> pattern.matcher(cacheName).matches()); - - return matchesInclude && notMatchesExclude; - } - - /** - * Finds match between cache name and user's regex templates. - * If match is found, adds this cache's id to id's list and saves cache name to file. - * - * @param cacheName Cache name. - */ - private void matchWithRegexTemplates(String cacheName) { + private void matchWithRegex(String cacheName) { int cacheId = CU.cacheId(cacheName); - if (!cachesIds.contains(cacheId) && matchesFilters(cacheName)) { + if (!cachesIds.contains(cacheId) && regexManager.match(cacheName)) cachesIds.add(cacheId); - - try { - List caches = loadCaches(); - - caches.add(cacheName); - - save(caches); - } - catch (IOException e) { - throw new IgniteException(e); - } - - if (log.isInfoEnabled()) - log.info("Cache has been added to replication [cacheName=" + cacheName + "]"); - } - } - - /** - * Writes caches list to file - * - * @param caches Caches list. - */ - private void save(List caches) throws IOException { - if (cdcDir == null) { - throw new IgniteException("Can't write to '" + SAVED_CACHES_FILE + "' file. Cdc directory is null"); - } - Path savedCachesPath = cdcDir.resolve(SAVED_CACHES_FILE); - Path tmpSavedCachesPath = cdcDir.resolve(SAVED_CACHES_TMP_FILE); - - StringBuilder cacheList = new StringBuilder(); - - for (String cache : caches) { - cacheList.append(cache); - - cacheList.append('\n'); - } - - Files.write(tmpSavedCachesPath, cacheList.toString().getBytes()); - - Files.move(tmpSavedCachesPath, savedCachesPath, ATOMIC_MOVE, REPLACE_EXISTING); } /** From a8493b12a4e389d140994efa0f0bcec259a8487b Mon Sep 17 00:00:00 2001 From: Andrei Nadyktov Date: Wed, 9 Jul 2025 15:58:06 +0300 Subject: [PATCH 09/14] IGNITE-22530 Add minor refactor --- .../java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java | 1 + .../org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java index c3867e241..66eec2504 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.binary.BinaryContext; import org.apache.ignite.internal.binary.BinaryMetadata; import org.apache.ignite.internal.binary.BinaryTypeImpl; +import org.apache.ignite.internal.cdc.CdcConsumerEx; import org.apache.ignite.internal.processors.metric.MetricRegistryImpl; import org.apache.ignite.internal.processors.metric.impl.AtomicLongMetric; import org.apache.ignite.internal.util.typedef.F; diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java index 6845a56cf..f24177f86 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java @@ -36,12 +36,12 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryType; import org.apache.ignite.cdc.CdcCacheEvent; -import org.apache.ignite.cdc.CdcConsumerEx; import org.apache.ignite.cdc.CdcEvent; import org.apache.ignite.cdc.CdcRegexManager; import org.apache.ignite.cdc.TypeMapping; import org.apache.ignite.cdc.conflictresolve.CacheVersionConflictResolverImpl; import org.apache.ignite.internal.binary.BinaryTypeImpl; +import org.apache.ignite.internal.cdc.CdcConsumerEx; import org.apache.ignite.internal.cdc.CdcMain; import org.apache.ignite.internal.processors.metric.MetricRegistryImpl; import org.apache.ignite.internal.processors.metric.impl.AtomicLongMetric; From d88d2ae7eaa34f85aaf5655ea977aa0553626315 Mon Sep 17 00:00:00 2001 From: Andrey Nadyktov Date: Tue, 5 Aug 2025 18:44:45 +0300 Subject: [PATCH 10/14] IGNITE-22530 Add Javadoc --- .../main/java/org/apache/ignite/cdc/CdcRegexManager.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java index 5d3cf7cf5..f6b6754f1 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java @@ -35,7 +35,7 @@ import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; /** - * TODO: Add JavaDoc + * Contains logic to process user's regexp patterns for CDC. */ public class CdcRegexManager implements CdcRegexMatcher { @@ -57,6 +57,11 @@ public class CdcRegexManager implements CdcRegexMatcher { /** Logger. */ private IgniteLogger log; + /** + * + * @param cdcDir Path to Change Data Capture Directory. + * @param log Logger. + */ public CdcRegexManager(Path cdcDir, IgniteLogger log) { this.cdcDir = cdcDir; this.log = log; From bf75beee57a0ff1e8ea6b9b54336db28568aef58 Mon Sep 17 00:00:00 2001 From: Andrey Nadyktov Date: Tue, 5 Aug 2025 22:03:58 +0300 Subject: [PATCH 11/14] IGNITE-22530 Fix indentation --- .../cdc/kafka/CdcKafkaReplicationTest.java | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java index 72e75ce81..da3f46ea3 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java @@ -281,26 +281,26 @@ private void checkK2IMetrics(Function longMetric) { * @return Future for Change Data Capture application. */ protected IgniteInternalFuture igniteToKafka( - IgniteConfiguration igniteCfg, - String topic, - String metadataTopic, - String cache, - Set includeTemplates, - Set excludeTemplates, - String threadName + IgniteConfiguration igniteCfg, + String topic, + String metadataTopic, + String cache, + Set includeTemplates, + Set excludeTemplates, + String threadName ) { return runAsync(() -> { IgniteToKafkaCdcStreamer cdcCnsmr = new IgniteToKafkaCdcStreamer() - .setTopic(topic) - .setMetadataTopic(metadataTopic) - .setKafkaPartitions(DFLT_PARTS) - .setCaches(Collections.singleton(cache)) - .setIncludeTemplates(includeTemplates) - .setExcludeTemplates(excludeTemplates) - .setMaxBatchSize(KEYS_CNT) - .setOnlyPrimary(false) - .setKafkaProperties(kafkaProperties()) - .setKafkaRequestTimeout(DFLT_KAFKA_REQ_TIMEOUT); + .setTopic(topic) + .setMetadataTopic(metadataTopic) + .setKafkaPartitions(DFLT_PARTS) + .setCaches(Collections.singleton(cache)) + .setIncludeTemplates(includeTemplates) + .setExcludeTemplates(excludeTemplates) + .setMaxBatchSize(KEYS_CNT) + .setOnlyPrimary(false) + .setKafkaProperties(kafkaProperties()) + .setKafkaRequestTimeout(DFLT_KAFKA_REQ_TIMEOUT); CdcConfiguration cdcCfg = new CdcConfiguration(); From 35afc131e348f0e161644dce730334322b8bf307 Mon Sep 17 00:00:00 2001 From: Andrei Nadyktov Date: Mon, 3 Nov 2025 20:47:04 +0300 Subject: [PATCH 12/14] IGNITE-22530 Remove usage of CdcRegexMatcher interface --- .../java/org/apache/ignite/cdc/CdcRegexManager.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java index f6b6754f1..d2a8e16cd 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java @@ -37,7 +37,7 @@ /** * Contains logic to process user's regexp patterns for CDC. */ -public class CdcRegexManager implements CdcRegexMatcher { +public class CdcRegexManager { /** File with saved names of caches added by cache masks. */ private static final String SAVED_CACHES_FILE = "caches"; @@ -67,8 +67,13 @@ public CdcRegexManager(Path cdcDir, IgniteLogger log) { this.log = log; } - /** {@inheritDoc} */ - @Override public boolean match(String cacheName) { + /** + * Finds and processes match between cache name and user's regexp patterns. + * + * @param cacheName Cache name. + * @return True if cache name matches user's regexp patterns. + */ + public boolean match(String cacheName) { return matchAndSave(cacheName); } From 654858900e81bdd7ab9ae4ff163a3c6138035036 Mon Sep 17 00:00:00 2001 From: Andrei Nadyktov Date: Mon, 3 Nov 2025 23:04:30 +0300 Subject: [PATCH 13/14] IGNITE-22530 WIP --- .../ignite/cdc/AbstractIgniteCdcStreamer.java | 27 +++++---- .../apache/ignite/cdc/CdcRegexManager.java | 35 ++++-------- .../cdc/kafka/IgniteToKafkaCdcStreamer.java | 27 +++++---- .../KafkaToIgniteCdcStreamerApplier.java | 20 +++---- ...KafkaToIgniteCdcStreamerConfiguration.java | 29 +++++----- .../ignite/cdc/AbstractReplicationTest.java | 14 ++--- .../cdc/CdcIgniteToIgniteReplicationTest.java | 30 +++++----- .../apache/ignite/cdc/RegexFiltersTest.java | 17 +++--- .../kafka/CdcKafkaReplicationAppsTest.java | 8 +-- .../cdc/kafka/CdcKafkaReplicationTest.java | 57 +++++++++---------- 10 files changed, 121 insertions(+), 143 deletions(-) diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java index 66eec2504..4fb6bc246 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java @@ -18,7 +18,6 @@ package org.apache.ignite.cdc; import java.nio.file.Path; -import java.util.HashSet; import java.util.Iterator; import java.util.Set; import java.util.stream.Collectors; @@ -80,11 +79,11 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx { /** Regexp manager. */ private CdcRegexManager regexManager; - /** Include regex templates for cache names. */ - private Set includeTemplates = new HashSet<>(); + /** Include regex template for cache names. */ + private String includeTemplate; - /** Exclude regex templates for cache names. */ - private Set excludeTemplates = new HashSet<>(); + /** Exclude regex template for cache names. */ + private String excludeTemplate; /** Cache IDs. */ protected Set cachesIds; @@ -127,7 +126,7 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx { .boxed() .collect(Collectors.toSet()); - regexManager.compileRegexp(includeTemplates, excludeTemplates); + regexManager.compileRegexp(includeTemplate, excludeTemplate); regexManager.getSavedCaches().stream() .map(CU::cacheId) @@ -276,25 +275,25 @@ public AbstractIgniteCdcStreamer setCaches(Set caches) { } /** - * Sets include regex patterns that participate in CDC. + * Sets include regex pattern that participate in CDC. * - * @param includeTemplates Include regex templates + * @param includeTemplate Include regex template * @return {@code this} for chaining. */ - public AbstractIgniteCdcStreamer setIncludeTemplates(Set includeTemplates) { - this.includeTemplates = includeTemplates; + public AbstractIgniteCdcStreamer setIncludeTemplate(String includeTemplate) { + this.includeTemplate = includeTemplate; return this; } /** - * Sets exclude regex patterns that participate in CDC. + * Sets exclude regex pattern that participate in CDC. * - * @param excludeTemplates Exclude regex templates + * @param excludeTemplate Exclude regex template * @return {@code this} for chaining. */ - public AbstractIgniteCdcStreamer setExcludeTemplates(Set excludeTemplates) { - this.excludeTemplates = excludeTemplates; + public AbstractIgniteCdcStreamer setExcludeTemplate(String excludeTemplate) { + this.excludeTemplate = excludeTemplate; return this; } diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java index d2a8e16cd..4e6bbbfac 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java @@ -22,7 +22,6 @@ import java.nio.file.Path; import java.util.List; import java.util.Optional; -import java.util.Set; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; import java.util.stream.Collectors; @@ -48,11 +47,11 @@ public class CdcRegexManager { /** CDC directory path. */ private final Path cdcDir; - /** Include regex patterns for cache names. */ - private Set includeFilters; + /** Include regex pattern for cache names. */ + private Pattern includeFilter; - /** Exclude regex patterns for cache names. */ - private Set excludeFilters; + /** Exclude regex pattern for cache names. */ + private Pattern excludeFilter; /** Logger. */ private IgniteLogger log; @@ -127,33 +126,23 @@ private boolean matchAndSave(String cacheName) { * Matches cache name with compiled regex patterns. * * @param cacheName Cache name. - * @return True if cache name matches include patterns and doesn't match exclude patterns. + * @return True if cache name matches include pattern and doesn't match exclude pattern. */ private boolean matchesFilters(String cacheName) { - boolean matchesInclude = includeFilters.stream() - .anyMatch(pattern -> pattern.matcher(cacheName).matches()); - - boolean notMatchesExclude = excludeFilters.stream() - .noneMatch(pattern -> pattern.matcher(cacheName).matches()); - - return matchesInclude && notMatchesExclude; + return includeFilter.matcher(cacheName).matches() && excludeFilter.matcher(cacheName).matches(); } /** * Compiles regex patterns from user templates. * - * @param includeTemplates Include regex templates. - * @param excludeTemplates Exclude regex templates. + * @param includeTemplate Include regex template. + * @param excludeTemplate Exclude regex template. * @throws PatternSyntaxException If the template's syntax is invalid */ - public void compileRegexp(Set includeTemplates, Set excludeTemplates) { - includeFilters = includeTemplates.stream() - .map(Pattern::compile) - .collect(Collectors.toSet()); - - excludeFilters = excludeTemplates.stream() - .map(Pattern::compile) - .collect(Collectors.toSet()); + public void compileRegexp(String includeTemplate, String excludeTemplate) { + includeFilter = Pattern.compile(includeTemplate); + + excludeFilter = Pattern.compile(excludeTemplate); } /** diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java index f24177f86..dd0f2c336 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java @@ -20,7 +20,6 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Collection; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Properties; @@ -154,11 +153,11 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumerEx { /** Regexp manager. */ private CdcRegexManager regexManager; - /** Include regex templates for cache names. */ - private Set includeTemplates = new HashSet<>(); + /** Include regex template for cache names. */ + private String includeTemplate; - /** Exclude regex templates for cache names. */ - private Set excludeTemplates = new HashSet<>(); + /** Exclude regex template for cache names. */ + private String excludeTemplate; /** Max batch size. */ private int maxBatchSz = DFLT_MAX_BATCH_SIZE; @@ -351,7 +350,7 @@ private void sendOneBatch( .map(CU::cacheId) .collect(Collectors.toSet()); - regexManager.compileRegexp(includeTemplates, excludeTemplates); + regexManager.compileRegexp(includeTemplate, excludeTemplate); regexManager.getSavedCaches().stream() .map(CU::cacheId) @@ -464,25 +463,25 @@ public IgniteToKafkaCdcStreamer setCaches(Collection caches) { } /** - * Sets include regex patterns that participate in CDC. + * Sets include regex pattern that participate in CDC. * - * @param includeTemplates Include regex templates. + * @param includeTemplate Include regex template. * @return {@code this} for chaining. */ - public IgniteToKafkaCdcStreamer setIncludeTemplates(Set includeTemplates) { - this.includeTemplates = includeTemplates; + public IgniteToKafkaCdcStreamer setIncludeTemplate(String includeTemplate) { + this.includeTemplate = includeTemplate; return this; } /** - * Sets exclude regex patterns that participate in CDC. + * Sets exclude regex pattern that participate in CDC. * - * @param excludeTemplates Exclude regex templates + * @param excludeTemplate Exclude regex template. * @return {@code this} for chaining. */ - public IgniteToKafkaCdcStreamer setExcludeTemplates(Set excludeTemplates) { - this.excludeTemplates = excludeTemplates; + public IgniteToKafkaCdcStreamer setExcludeTemplate(String excludeTemplate) { + this.excludeTemplate = excludeTemplate; return this; } diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java index 4b67189c4..e543b91f6 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java @@ -108,11 +108,11 @@ class KafkaToIgniteCdcStreamerApplier implements Runnable, AutoCloseable { /** Caches ids to read. */ private final Set caches; - /** Include regex templates for cache names. */ - private final Set includeTemplates; + /** Include regex template for cache names. */ + private final String includeTemplate; - /** Exclude regex templates for cache names. */ - private final Set excludeTemplates; + /** Exclude regex template for cache names. */ + private final String excludeTemplate; /** The maximum time to complete Kafka related requests, in milliseconds. */ private final long kafkaReqTimeout; @@ -180,8 +180,8 @@ public KafkaToIgniteCdcStreamerApplier( this.log = log.getLogger(KafkaToIgniteCdcStreamerApplier.class); this.metrics = metrics; this.streamer = streamer; - this.includeTemplates = streamerCfg.getIncludeTemplates(); - this.excludeTemplates = streamerCfg.getExcludeTemplates(); + this.includeTemplate = streamerCfg.getIncludeTemplate(); + this.excludeTemplate = streamerCfg.getExcludeTemplate(); } /** {@inheritDoc} */ @@ -307,13 +307,9 @@ private boolean matchesRegexTemplates(Integer key) { * @return True if cache name match include patterns and don't match exclude patterns. */ private boolean matchesFilters(String cacheName) { - boolean matchesInclude = includeTemplates.stream() - .map(Pattern::compile) - .anyMatch(pattern -> pattern.matcher(cacheName).matches()); + boolean matchesInclude = Pattern.compile(includeTemplate).matcher(cacheName).matches(); - boolean notMatchesExclude = excludeTemplates.stream() - .map(Pattern::compile) - .noneMatch(pattern -> pattern.matcher(cacheName).matches()); + boolean notMatchesExclude = Pattern.compile(excludeTemplate).matcher(cacheName).matches(); return matchesInclude && notMatchesExclude; } diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java index ece45402e..2a2a8e38e 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java @@ -18,7 +18,6 @@ package org.apache.ignite.cdc.kafka; import java.util.Collection; -import java.util.HashSet; import java.util.Map; import java.util.Set; @@ -86,11 +85,11 @@ public class KafkaToIgniteCdcStreamerConfiguration { */ private Collection caches; - /** Include regex templates for cache names. */ - private Set includeTemplates = new HashSet<>(); + /** Include regex template for cache names. */ + private String includeTemplate; - /** Exclude regex templates for cache names. */ - private Set excludeTemplates = new HashSet<>(); + /** Exclude regex template for cache names. */ + private String excludeTemplate; /** Metric exporter SPI. */ private MetricExporterSpi[] metricExporterSpi; @@ -185,29 +184,29 @@ public void setCaches(Collection caches) { /** * @return Include regex templates */ - public Set getIncludeTemplates() { - return includeTemplates; + public String getIncludeTemplate() { + return includeTemplate; } /** - * @param includeTemplates Include regex templates + * @param includeTemplate Include regex templates */ - public void setIncludeTemplates(Set includeTemplates) { - this.includeTemplates = includeTemplates; + public void setIncludeTemplate(String includeTemplate) { + this.includeTemplate = includeTemplate; } /** * @return Exclude regex templates */ - public Set getExcludeTemplates() { - return excludeTemplates; + public String getExcludeTemplate() { + return excludeTemplate; } /** - * @param excludeTemplates Exclude regex templates + * @param excludeTemplate Exclude regex templates */ - public void setExcludeTemplates(Set excludeTemplates) { - this.excludeTemplates = excludeTemplates; + public void setExcludeTemplate(String excludeTemplate) { + this.excludeTemplate = excludeTemplate; } /** @return The maximum time to complete Kafka related requests, in milliseconds. */ diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java index c79657f76..859744bb1 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java @@ -599,7 +599,7 @@ public void testActiveActiveReplicationWithRegexFilters() throws Exception { IntStream.range(0, KEYS_CNT).filter(i -> i % 2 != 0))); //Start CDC with only 'active-active-cache' in 'caches' property of CDC config - List> futs = startActiveActiveCdcWithFilters(includeTemplates, excludeTemplates); + List> futs = startActiveActiveCdcWithFilters(REGEX_INCLUDE_PATTERN, REGEX_EXCLUDE_PATTERN); try { waitForSameData(srcCache, destCache, KEYS_CNT, WaitDataMode.EXISTS, futs); @@ -628,8 +628,8 @@ public void testActivePassiveReplicationWithRegexFilters() throws Exception { Set excludeTemplates = new HashSet<>(Arrays.asList(REGEX_EXCLUDE_PATTERN)); //Start CDC with only 'active-active-cache' in 'caches' property of CDC config - List> futs = startActivePassiveCdcWithFilters(ACTIVE_PASSIVE_CACHE, - includeTemplates, excludeTemplates); + List> futs = startActivePassiveCdcWithFilters(ACTIVE_PASSIVE_CACHE, REGEX_INCLUDE_PATTERN, + REGEX_EXCLUDE_PATTERN); try { createCache(destCluster[0], ACTIVE_PASSIVE_CACHE); @@ -797,15 +797,15 @@ protected String[] hostAddresses(IgniteEx[] dest) { /** */ protected abstract List> startActivePassiveCdcWithFilters(String cache, - Set includeTemplates, - Set excludeTemplates); + String includeTemplate, + String excludeTemplate); /** */ protected abstract List> startActiveActiveCdc(); /** */ - protected abstract List> startActiveActiveCdcWithFilters(Set includeTemplates, - Set excludeTemplates); + protected abstract List> startActiveActiveCdcWithFilters(String includeTemplate, + String excludeTemplate); /** */ protected abstract void checkConsumerMetrics(Function longMetric); diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java index 9c021645c..70994c34b 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java @@ -46,40 +46,40 @@ public class CdcIgniteToIgniteReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActivePassiveCdc(String cache) { - return startActivePassiveCdcWithFilters(cache, Collections.emptySet(), Collections.emptySet()); + return startActivePassiveCdcWithFilters(cache, null, null); } /** {@inheritDoc} */ @Override protected List> startActivePassiveCdcWithFilters(String cache, - Set includeTemplates, - Set excludeTemplates) { + String includeTemplate, + String excludeTemplate) { List> futs = new ArrayList<>(); for (int i = 0; i < srcCluster.length; i++) futs.add(igniteToIgnite(srcCluster[i].configuration(), destClusterCliCfg[i], destCluster, cache, - includeTemplates, excludeTemplates, "ignite-to-ignite-src-" + i)); + includeTemplate, excludeTemplate, "ignite-to-ignite-src-" + i)); return futs; } /** {@inheritDoc} */ @Override protected List> startActiveActiveCdc() { - return startActiveActiveCdcWithFilters(Collections.emptySet(), Collections.emptySet()); + return startActiveActiveCdcWithFilters(null, null); } /** {@inheritDoc} */ - @Override protected List> startActiveActiveCdcWithFilters(Set includeTemplates, - Set excludeTemplates) { + @Override protected List> startActiveActiveCdcWithFilters(String includeTemplate, + String excludeTemplate) { List> futs = new ArrayList<>(); for (int i = 0; i < srcCluster.length; i++) { futs.add(igniteToIgnite(srcCluster[i].configuration(), destClusterCliCfg[i], destCluster, - ACTIVE_ACTIVE_CACHE, includeTemplates, excludeTemplates, "ignite-to-ignite-src-" + i)); + ACTIVE_ACTIVE_CACHE, includeTemplate, excludeTemplate, "ignite-to-ignite-src-" + i)); } for (int i = 0; i < destCluster.length; i++) { futs.add(igniteToIgnite(destCluster[i].configuration(), srcClusterCliCfg[i], srcCluster, - ACTIVE_ACTIVE_CACHE, includeTemplates, excludeTemplates, "ignite-to-ignite-dest-" + i)); + ACTIVE_ACTIVE_CACHE, includeTemplate, excludeTemplate, "ignite-to-ignite-dest-" + i)); } return futs; @@ -101,8 +101,8 @@ public class CdcIgniteToIgniteReplicationTest extends AbstractReplicationTest { * @param destCfg Ignite destination cluster configuration. * @param dest Ignite destination cluster. * @param cache Cache name to stream to kafka. - * @param includeTemplates Include regex templates for cache names. - * @param excludeTemplates Exclude regex templates for cache names. + * @param includeTemplate Include regex templates for cache names. + * @param excludeTemplate Exclude regex templates for cache names. * @param threadName Thread to run CDC instance. * @return Future for Change Data Capture application. */ @@ -111,8 +111,8 @@ protected IgniteInternalFuture igniteToIgnite( IgniteConfiguration destCfg, IgniteEx[] dest, String cache, - Set includeTemplates, - Set excludeTemplates, + String includeTemplate, + String excludeTemplate, @Nullable String threadName ) { return runAsync(() -> { @@ -134,8 +134,8 @@ protected IgniteInternalFuture igniteToIgnite( streamer.setMaxBatchSize(KEYS_CNT); streamer.setCaches(Collections.singleton(cache)); - streamer.setIncludeTemplates(includeTemplates); - streamer.setExcludeTemplates(excludeTemplates); + streamer.setIncludeTemplate(includeTemplate); + streamer.setExcludeTemplate(excludeTemplate); cdcCfg.setConsumer(streamer); cdcCfg.setMetricExporterSpi(new JmxMetricExporterSpi()); diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersTest.java index e4007ab08..a30e129b1 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersTest.java @@ -64,7 +64,7 @@ private enum WaitDataMode { private static final String REGEX_INCLUDE_PATTERN = "regex.*"; /** */ - private Set includeTemplates; + private String includeTemplates; /** */ private static final int KEYS_CNT = 1000; @@ -96,14 +96,13 @@ private enum WaitDataMode { * * @param srcCfg Ignite source node configuration. * @param cache Cache name to stream to Ignite2Ignite. - * @param includeTemplates Include cache templates. + * @param includeTemplate Include cache templates. * @param excludeTemplates Exclude cache templates. * @return Future for Change Data Capture application. */ private IgniteInternalFuture startCdc(IgniteConfiguration srcCfg, String cache, - Set includeTemplates, - Set excludeTemplates) { + String includeTemplate) { return runAsync(() -> { CdcConfiguration cdcCfg = new CdcConfiguration(); @@ -114,8 +113,8 @@ private IgniteInternalFuture startCdc(IgniteConfiguration srcCfg, streamer.setMaxBatchSize(KEYS_CNT); streamer.setCaches(Collections.singleton(cache)); - streamer.setIncludeTemplates(includeTemplates); - streamer.setExcludeTemplates(excludeTemplates); + streamer.setIncludeTemplate(includeTemplate); + streamer.setExcludeTemplate(""); cdcCfg.setConsumer(streamer); cdcCfg.setMetricExporterSpi(new JmxMetricExporterSpi()); @@ -135,8 +134,6 @@ private IgniteInternalFuture startCdc(IgniteConfiguration srcCfg, discoPort += DFLT_PORT_RANGE + 1; dest = startGrid(getConfiguration("dest-cluster")); - - includeTemplates = new HashSet<>(Arrays.asList(REGEX_INCLUDE_PATTERN)); } /** {@inheritDoc} */ @@ -204,7 +201,7 @@ public void testRegexFiltersOnCdcRestart() throws Exception { dest.cluster().state(ClusterState.ACTIVE); //Start CDC only with 'test-cache' in config and cache masks (regex filters) - IgniteInternalFuture cdc = startCdc(src.configuration(), TEST_CACHE, includeTemplates, Collections.emptySet()); + IgniteInternalFuture cdc = startCdc(src.configuration(), TEST_CACHE, REGEX_INCLUDE_PATTERN); IgniteCache srcCache = src.getOrCreateCache(new CacheConfiguration() .setName(REGEX_MATCHING_CACHE) @@ -219,7 +216,7 @@ public void testRegexFiltersOnCdcRestart() throws Exception { cdc.cancel(); //Restart CDC - IgniteInternalFuture cdc2 = startCdc(src.configuration(), TEST_CACHE, includeTemplates, Collections.emptySet()); + IgniteInternalFuture cdc2 = startCdc(src.configuration(), TEST_CACHE, REGEX_INCLUDE_PATTERN); try { runAsync(generateData(srcCache, IntStream.range(0, KEYS_CNT))); diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java index e2f20df1a..d00ebb82e 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java @@ -114,8 +114,8 @@ public class CdcKafkaReplicationAppsTest extends CdcKafkaReplicationTest { String topic, String metadataTopic, String cache, - Set includeTemplates, - Set excludeTemplates, + String includeTemplate, + String excludeTemplate, String threadName ) { Map params = new HashMap<>(); @@ -144,8 +144,8 @@ public class CdcKafkaReplicationAppsTest extends CdcKafkaReplicationTest { IgniteEx[] dest, int partFrom, int partTo, - Set includeTemplates, - Set excludeTemplates, + String includeTemplate, + String excludeTemplate, String threadName ) { Map params = new HashMap<>(); diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java index da3f46ea3..f68d64c3c 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java @@ -21,7 +21,6 @@ import java.util.Collections; import java.util.List; import java.util.Properties; -import java.util.Set; import java.util.function.Function; import java.util.function.Supplier; import javax.management.DynamicMBean; @@ -94,13 +93,13 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActivePassiveCdc(String cache) { - return startActivePassiveCdcWithFilters(cache, Collections.emptySet(), Collections.emptySet()); + return startActivePassiveCdcWithFilters(cache, null, null); } /** {@inheritDoc} */ @Override protected List> startActivePassiveCdcWithFilters(String cache, - Set includeTemplates, - Set excludeTemplates) { + String includeTemplate, + String excludeTemplate) { try { KAFKA.createTopic(cache, DFLT_PARTS, 1); @@ -115,8 +114,8 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { for (IgniteEx ex : srcCluster) { int idx = getTestIgniteInstanceIndex(ex.name()); - futs.add(igniteToKafka(ex.configuration(), cache, SRC_DEST_META_TOPIC, cache, includeTemplates, - excludeTemplates, "ignite-src-to-kafka-" + idx)); + futs.add(igniteToKafka(ex.configuration(), cache, SRC_DEST_META_TOPIC, cache, includeTemplate, + excludeTemplate, "ignite-src-to-kafka-" + idx)); } for (int i = 0; i < destCluster.length; i++) { @@ -128,8 +127,8 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { destCluster, i * (DFLT_PARTS / 2), (i + 1) * (DFLT_PARTS / 2), - includeTemplates, - excludeTemplates, + includeTemplate, + excludeTemplate, "kafka-to-ignite-dest-" + i )); } @@ -139,26 +138,26 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActiveActiveCdc() { - return startActiveActiveCdcWithFilters(Collections.emptySet(), Collections.emptySet()); + return startActiveActiveCdcWithFilters(null, null); } /** {@inheritDoc} */ - @Override protected List> startActiveActiveCdcWithFilters(Set includeTemplates, - Set excludeTemplates) { + @Override protected List> startActiveActiveCdcWithFilters(String includeTemplate, + String excludeTemplate) { List> futs = new ArrayList<>(); for (IgniteEx ex : srcCluster) { int idx = getTestIgniteInstanceIndex(ex.name()); futs.add(igniteToKafka(ex.configuration(), SRC_DEST_TOPIC, SRC_DEST_META_TOPIC, ACTIVE_ACTIVE_CACHE, - includeTemplates, excludeTemplates, "ignite-src-to-kafka-" + idx)); + includeTemplate, excludeTemplate, "ignite-src-to-kafka-" + idx)); } for (IgniteEx ex : destCluster) { int idx = getTestIgniteInstanceIndex(ex.name()); futs.add(igniteToKafka(ex.configuration(), DEST_SRC_TOPIC, DEST_SRC_META_TOPIC, ACTIVE_ACTIVE_CACHE, - includeTemplates, excludeTemplates, "ignite-dest-to-kafka-" + idx)); + includeTemplate, excludeTemplate, "ignite-dest-to-kafka-" + idx)); } futs.add(kafkaToIgnite( @@ -169,8 +168,8 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { destCluster, 0, DFLT_PARTS, - includeTemplates, - excludeTemplates, + includeTemplate, + excludeTemplate, "kafka-to-ignite-src" )); @@ -182,8 +181,8 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { srcCluster, 0, DFLT_PARTS, - includeTemplates, - excludeTemplates, + includeTemplate, + excludeTemplate, "kafka-to-ignite-dest" )); @@ -276,8 +275,8 @@ private void checkK2IMetrics(Function longMetric) { * @param topic Kafka topic name. * @param metadataTopic Metadata topic name. * @param cache Cache name to stream to kafka. - * @param includeTemplates Include regex templates for cache names. - * @param excludeTemplates Exclude regex templates for cache names. + * @param includeTemplate Include regex templates for cache names. + * @param excludeTemplate Exclude regex templates for cache names. * @return Future for Change Data Capture application. */ protected IgniteInternalFuture igniteToKafka( @@ -285,8 +284,8 @@ protected IgniteInternalFuture igniteToKafka( String topic, String metadataTopic, String cache, - Set includeTemplates, - Set excludeTemplates, + String includeTemplate, + String excludeTemplate, String threadName ) { return runAsync(() -> { @@ -295,8 +294,8 @@ protected IgniteInternalFuture igniteToKafka( .setMetadataTopic(metadataTopic) .setKafkaPartitions(DFLT_PARTS) .setCaches(Collections.singleton(cache)) - .setIncludeTemplates(includeTemplates) - .setExcludeTemplates(excludeTemplates) + .setIncludeTemplate(includeTemplate) + .setExcludeTemplate(excludeTemplate) .setMaxBatchSize(KEYS_CNT) .setOnlyPrimary(false) .setKafkaProperties(kafkaProperties()) @@ -319,8 +318,8 @@ protected IgniteInternalFuture igniteToKafka( * @param cacheName Cache name. * @param igniteCfg Ignite configuration. * @param dest Destination Ignite cluster. - * @param includeTemplates Include regex templates for cache names. - * @param excludeTemplates Exclude regex templates for cache names. + * @param includeTemplate Include regex templates for cache names. + * @param excludeTemplate Exclude regex templates for cache names. * @return Future for runed {@link KafkaToIgniteCdcStreamer}. */ protected IgniteInternalFuture kafkaToIgnite( @@ -331,8 +330,8 @@ protected IgniteInternalFuture kafkaToIgnite( IgniteEx[] dest, int fromPart, int toPart, - Set includeTemplates, - Set excludeTemplates, + String includeTemplate, + String excludeTemplate, String threadName ) { KafkaToIgniteCdcStreamerConfiguration cfg = new KafkaToIgniteCdcStreamerConfiguration(); @@ -342,8 +341,8 @@ protected IgniteInternalFuture kafkaToIgnite( cfg.setThreadCount((toPart - fromPart) / 2); cfg.setCaches(Collections.singletonList(cacheName)); - cfg.setIncludeTemplates(includeTemplates); - cfg.setExcludeTemplates(excludeTemplates); + cfg.setIncludeTemplate(includeTemplate); + cfg.setExcludeTemplate(excludeTemplate); cfg.setTopic(topic); cfg.setMetadataTopic(metadataTopic); cfg.setKafkaRequestTimeout(DFLT_KAFKA_REQ_TIMEOUT); From fcfe3fd8132f7aadb5a075f28b6337f69accaa46 Mon Sep 17 00:00:00 2001 From: Andrey Nadyktov Date: Sat, 8 Nov 2025 20:06:07 +0300 Subject: [PATCH 14/14] IGNITE-22530 Make each regex pattern a single string --- .../ignite/cdc/AbstractIgniteCdcStreamer.java | 11 +++-- .../apache/ignite/cdc/CdcRegexManager.java | 13 ++++-- ...VersionConflictResolverPluginProvider.java | 41 +++++++++++-------- .../cdc/kafka/IgniteToKafkaCdcStreamer.java | 11 +++-- .../KafkaToIgniteCdcStreamerApplier.java | 15 +++++-- ...KafkaToIgniteCdcStreamerConfiguration.java | 9 ++-- .../ignite/cdc/AbstractReplicationTest.java | 11 +---- .../cdc/CdcIgniteToIgniteReplicationTest.java | 9 ++-- ...ersTest.java => RegexFiltersSelfTest.java} | 18 +++----- .../kafka/CdcKafkaReplicationAppsTest.java | 1 - .../cdc/kafka/CdcKafkaReplicationTest.java | 12 +++--- 11 files changed, 79 insertions(+), 72 deletions(-) rename modules/cdc-ext/src/test/java/org/apache/ignite/cdc/{RegexFiltersTest.java => RegexFiltersSelfTest.java} (93%) diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java index 4fb6bc246..8b32b24cd 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/AbstractIgniteCdcStreamer.java @@ -70,6 +70,9 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx { /** */ public static final String LAST_EVT_SENT_TIME_DESC = "Timestamp of last applied event to destination cluster"; + /** */ + public static final String DFLT_REGEXP = ""; + /** Handle only primary entry flag. */ private boolean onlyPrimary = DFLT_IS_ONLY_PRIMARY; @@ -80,10 +83,10 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx { private CdcRegexManager regexManager; /** Include regex template for cache names. */ - private String includeTemplate; + private String includeTemplate = DFLT_REGEXP; /** Exclude regex template for cache names. */ - private String excludeTemplate; + private String excludeTemplate = DFLT_REGEXP; /** Cache IDs. */ protected Set cachesIds; @@ -275,7 +278,7 @@ public AbstractIgniteCdcStreamer setCaches(Set caches) { } /** - * Sets include regex pattern that participate in CDC. + * Sets include regex pattern that participates in CDC. * * @param includeTemplate Include regex template * @return {@code this} for chaining. @@ -287,7 +290,7 @@ public AbstractIgniteCdcStreamer setIncludeTemplate(String includeTemplate) { } /** - * Sets exclude regex pattern that participate in CDC. + * Sets exclude regex pattern that participates in CDC. * * @param excludeTemplate Exclude regex template * @return {@code this} for chaining. diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java index 4e6bbbfac..a099d08ab 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java @@ -129,7 +129,7 @@ private boolean matchAndSave(String cacheName) { * @return True if cache name matches include pattern and doesn't match exclude pattern. */ private boolean matchesFilters(String cacheName) { - return includeFilter.matcher(cacheName).matches() && excludeFilter.matcher(cacheName).matches(); + return includeFilter.matcher(cacheName).matches() && !excludeFilter.matcher(cacheName).matches(); } /** @@ -137,12 +137,17 @@ private boolean matchesFilters(String cacheName) { * * @param includeTemplate Include regex template. * @param excludeTemplate Exclude regex template. - * @throws PatternSyntaxException If the template's syntax is invalid + * @throws IgniteException If the template's syntax is invalid */ public void compileRegexp(String includeTemplate, String excludeTemplate) { - includeFilter = Pattern.compile(includeTemplate); + try { + includeFilter = Pattern.compile(includeTemplate); - excludeFilter = Pattern.compile(excludeTemplate); + excludeFilter = Pattern.compile(excludeTemplate); + } + catch (PatternSyntaxException e) { + throw new IgniteException("Invalid cache regexp template.", e); + } } /** diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/conflictresolve/CacheVersionConflictResolverPluginProvider.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/conflictresolve/CacheVersionConflictResolverPluginProvider.java index f283b6c2c..b87157705 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/conflictresolve/CacheVersionConflictResolverPluginProvider.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/conflictresolve/CacheVersionConflictResolverPluginProvider.java @@ -18,11 +18,12 @@ package org.apache.ignite.cdc.conflictresolve; import java.io.Serializable; -import java.util.HashSet; import java.util.Set; import java.util.UUID; import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteEx; @@ -44,6 +45,9 @@ * @see CacheVersionConflictResolver */ public class CacheVersionConflictResolverPluginProvider implements PluginProvider { + /** */ + public static final String DFLT_REGEXP = ""; + /** Cluster id. */ private byte clusterId; @@ -69,10 +73,10 @@ public class CacheVersionConflictResolverPluginProvider includeTemplates = new HashSet<>(); + private String includeTemplate = DFLT_REGEXP; /** Exclude regex templates for cache names. */ - private Set excludeTemplates = new HashSet<>(); + private String excludeTemplate = DFLT_REGEXP; /** Log. */ private IgniteLogger log; @@ -153,14 +157,14 @@ public void setConflictResolver(CacheVersionConflictResolver resolver) { this.resolver = resolver; } - /** @param includeTemplates Include regex templates */ - public void setIncludeTemplates(Set includeTemplates) { - this.includeTemplates = includeTemplates; + /** @param includeTemplate Include regex template */ + public void setIncludeTemplate(String includeTemplate) { + this.includeTemplate = includeTemplate; } - /** @param excludeTemplates Exclude regex templates */ - public void setExcludeTemplates(Set excludeTemplates) { - this.excludeTemplates = excludeTemplates; + /** @param excludeTemplate Exclude regex template */ + public void setExcludeTemplate(String excludeTemplate) { + this.excludeTemplate = excludeTemplate; } /** {@inheritDoc} */ @@ -199,19 +203,22 @@ public void setExcludeTemplates(Set excludeTemplates) { } /** - * Match cache name with regex patterns. + * Matches cache name with compiled regex patterns. * * @param cacheName Cache name. + * @return True if cache name matches include pattern and doesn't match exclude pattern. + * @throws IgniteException If the template's syntax is invalid */ private boolean matchesFilters(String cacheName) { - boolean matchesInclude = includeTemplates.stream() - .map(Pattern::compile) - .anyMatch(pattern -> pattern.matcher(cacheName).matches()); + try { + boolean matchesInclude = Pattern.compile(includeTemplate).matcher(cacheName).matches(); - boolean notMatchesExclude = excludeTemplates.stream() - .map(Pattern::compile) - .noneMatch(pattern -> pattern.matcher(cacheName).matches()); + boolean matchesExclude = Pattern.compile(excludeTemplate).matcher(cacheName).matches(); - return matchesInclude && notMatchesExclude; + return matchesInclude && !matchesExclude; + } + catch (PatternSyntaxException e) { + throw new IgniteException("Invalid cache regexp template.", e); + } } } diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java index dd0f2c336..b60ec1431 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/IgniteToKafkaCdcStreamer.java @@ -116,6 +116,9 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumerEx { /** Count of metadata markers sent description. */ public static final String MARKERS_SENT_CNT_DESC = "Count of metadata markers sent to Kafka"; + /** */ + public static final String DFLT_REGEXP = ""; + /** Default value for the flag that indicates whether entries only from primary nodes should be handled. */ public static final boolean DFLT_IS_ONLY_PRIMARY = false; @@ -154,10 +157,10 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumerEx { private CdcRegexManager regexManager; /** Include regex template for cache names. */ - private String includeTemplate; + private String includeTemplate = DFLT_REGEXP; /** Exclude regex template for cache names. */ - private String excludeTemplate; + private String excludeTemplate = DFLT_REGEXP; /** Max batch size. */ private int maxBatchSz = DFLT_MAX_BATCH_SIZE; @@ -463,7 +466,7 @@ public IgniteToKafkaCdcStreamer setCaches(Collection caches) { } /** - * Sets include regex pattern that participate in CDC. + * Sets include regex pattern that participates in CDC. * * @param includeTemplate Include regex template. * @return {@code this} for chaining. @@ -475,7 +478,7 @@ public IgniteToKafkaCdcStreamer setIncludeTemplate(String includeTemplate) { } /** - * Sets exclude regex pattern that participate in CDC. + * Sets exclude regex pattern that participates in CDC. * * @param excludeTemplate Exclude regex template. * @return {@code this} for chaining. diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java index e543b91f6..200ae59c2 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java @@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -304,14 +305,20 @@ private boolean matchesRegexTemplates(Integer key) { * Matches cache name with compiled regex patterns. * * @param cacheName Cache name. - * @return True if cache name match include patterns and don't match exclude patterns. + * @return True if cache name matches include pattern and doesn't match exclude pattern. + * @throws IgniteException If the template's syntax is invalid */ private boolean matchesFilters(String cacheName) { - boolean matchesInclude = Pattern.compile(includeTemplate).matcher(cacheName).matches(); + try { + boolean matchesInclude = Pattern.compile(includeTemplate).matcher(cacheName).matches(); - boolean notMatchesExclude = Pattern.compile(excludeTemplate).matcher(cacheName).matches(); + boolean matchesExclude = Pattern.compile(excludeTemplate).matcher(cacheName).matches(); - return matchesInclude && notMatchesExclude; + return matchesInclude && !matchesExclude; + } + catch (PatternSyntaxException e) { + throw new IgniteException("Invalid cache regexp template.", e); + } } /** diff --git a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java index 2a2a8e38e..683d6d461 100644 --- a/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerConfiguration.java @@ -19,7 +19,6 @@ import java.util.Collection; import java.util.Map; -import java.util.Set; import org.apache.ignite.cdc.CdcConfiguration; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; @@ -182,28 +181,28 @@ public void setCaches(Collection caches) { } /** - * @return Include regex templates + * @return Include regex template. */ public String getIncludeTemplate() { return includeTemplate; } /** - * @param includeTemplate Include regex templates + * @param includeTemplate Include regex template. */ public void setIncludeTemplate(String includeTemplate) { this.includeTemplate = includeTemplate; } /** - * @return Exclude regex templates + * @return Exclude regex template */ public String getExcludeTemplate() { return excludeTemplate; } /** - * @param excludeTemplate Exclude regex templates + * @param excludeTemplate Exclude regex template. */ public void setExcludeTemplate(String excludeTemplate) { this.excludeTemplate = excludeTemplate; diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java index 859744bb1..529e1ebcb 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/AbstractReplicationTest.java @@ -25,7 +25,6 @@ import java.util.EnumSet; import java.util.HashSet; import java.util.List; -import java.util.Set; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; @@ -213,8 +212,8 @@ private enum WaitDataMode { cfgPlugin1.setClusterId(clusterId); cfgPlugin1.setCaches(new HashSet<>(Arrays.asList(ACTIVE_PASSIVE_CACHE, ACTIVE_ACTIVE_CACHE))); - cfgPlugin1.setIncludeTemplates(new HashSet<>(Arrays.asList(REGEX_INCLUDE_PATTERN))); - cfgPlugin1.setExcludeTemplates(new HashSet<>(Arrays.asList(REGEX_EXCLUDE_PATTERN))); + cfgPlugin1.setIncludeTemplate(REGEX_INCLUDE_PATTERN); + cfgPlugin1.setExcludeTemplate(REGEX_EXCLUDE_PATTERN); cfgPlugin1.setConflictResolveField("reqId"); cfg.setPluginProviders(cfgPlugin1); @@ -581,9 +580,6 @@ public void testWithExpiryPolicy() throws Exception { * Active/Active mode means changes made in both clusters. */ @Test public void testActiveActiveReplicationWithRegexFilters() throws Exception { - Set includeTemplates = new HashSet<>(Arrays.asList(REGEX_INCLUDE_PATTERN)); - Set excludeTemplates = new HashSet<>(Arrays.asList(REGEX_EXCLUDE_PATTERN)); - createCache(srcCluster[0], ACTIVE_ACTIVE_CACHE); createCache(destCluster[0], ACTIVE_ACTIVE_CACHE); @@ -624,9 +620,6 @@ public void testActiveActiveReplicationWithRegexFilters() throws Exception { * Active/Passive mode means changes made only in one cluster. */ @Test public void testActivePassiveReplicationWithRegexFilters() throws Exception { - Set includeTemplates = new HashSet<>(Arrays.asList(REGEX_INCLUDE_PATTERN)); - Set excludeTemplates = new HashSet<>(Arrays.asList(REGEX_EXCLUDE_PATTERN)); - //Start CDC with only 'active-active-cache' in 'caches' property of CDC config List> futs = startActivePassiveCdcWithFilters(ACTIVE_PASSIVE_CACHE, REGEX_INCLUDE_PATTERN, REGEX_EXCLUDE_PATTERN); diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java index 70994c34b..bab982936 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/CdcIgniteToIgniteReplicationTest.java @@ -20,7 +20,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Set; import java.util.function.Function; import org.apache.ignite.Ignition; import org.apache.ignite.cdc.thin.IgniteToIgniteClientCdcStreamer; @@ -46,7 +45,7 @@ public class CdcIgniteToIgniteReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActivePassiveCdc(String cache) { - return startActivePassiveCdcWithFilters(cache, null, null); + return startActivePassiveCdcWithFilters(cache, "", ""); } /** {@inheritDoc} */ @@ -64,7 +63,7 @@ public class CdcIgniteToIgniteReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActiveActiveCdc() { - return startActiveActiveCdcWithFilters(null, null); + return startActiveActiveCdcWithFilters("", ""); } /** {@inheritDoc} */ @@ -101,8 +100,8 @@ public class CdcIgniteToIgniteReplicationTest extends AbstractReplicationTest { * @param destCfg Ignite destination cluster configuration. * @param dest Ignite destination cluster. * @param cache Cache name to stream to kafka. - * @param includeTemplate Include regex templates for cache names. - * @param excludeTemplate Exclude regex templates for cache names. + * @param includeTemplate Include regex template for cache names. + * @param excludeTemplate Exclude regex template for cache names. * @param threadName Thread to run CDC instance. * @return Future for Change Data Capture application. */ diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersSelfTest.java similarity index 93% rename from modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersTest.java rename to modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersSelfTest.java index a30e129b1..1d31dfc9a 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersSelfTest.java @@ -1,9 +1,6 @@ package org.apache.ignite.cdc; -import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; -import java.util.Set; import java.util.stream.IntStream; import org.apache.ignite.IgniteCache; @@ -34,7 +31,7 @@ import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** */ -public class RegexFiltersTest extends GridCommonAbstractTest { +public class RegexFiltersSelfTest extends GridCommonAbstractTest { /** */ private IgniteEx src; @@ -61,10 +58,7 @@ private enum WaitDataMode { private static final String REGEX_MATCHING_CACHE = "regex-cache"; /** */ - private static final String REGEX_INCLUDE_PATTERN = "regex.*"; - - /** */ - private String includeTemplates; + private static final String REGEX_PATTERN = "regex.*"; /** */ private static final int KEYS_CNT = 1000; @@ -96,8 +90,7 @@ private enum WaitDataMode { * * @param srcCfg Ignite source node configuration. * @param cache Cache name to stream to Ignite2Ignite. - * @param includeTemplate Include cache templates. - * @param excludeTemplates Exclude cache templates. + * @param includeTemplate Include cache template. * @return Future for Change Data Capture application. */ private IgniteInternalFuture startCdc(IgniteConfiguration srcCfg, @@ -114,7 +107,6 @@ private IgniteInternalFuture startCdc(IgniteConfiguration srcCfg, streamer.setMaxBatchSize(KEYS_CNT); streamer.setCaches(Collections.singleton(cache)); streamer.setIncludeTemplate(includeTemplate); - streamer.setExcludeTemplate(""); cdcCfg.setConsumer(streamer); cdcCfg.setMetricExporterSpi(new JmxMetricExporterSpi()); @@ -201,7 +193,7 @@ public void testRegexFiltersOnCdcRestart() throws Exception { dest.cluster().state(ClusterState.ACTIVE); //Start CDC only with 'test-cache' in config and cache masks (regex filters) - IgniteInternalFuture cdc = startCdc(src.configuration(), TEST_CACHE, REGEX_INCLUDE_PATTERN); + IgniteInternalFuture cdc = startCdc(src.configuration(), TEST_CACHE, REGEX_PATTERN); IgniteCache srcCache = src.getOrCreateCache(new CacheConfiguration() .setName(REGEX_MATCHING_CACHE) @@ -216,7 +208,7 @@ public void testRegexFiltersOnCdcRestart() throws Exception { cdc.cancel(); //Restart CDC - IgniteInternalFuture cdc2 = startCdc(src.configuration(), TEST_CACHE, REGEX_INCLUDE_PATTERN); + IgniteInternalFuture cdc2 = startCdc(src.configuration(), TEST_CACHE, REGEX_PATTERN); try { runAsync(generateData(srcCache, IntStream.range(0, KEYS_CNT))); diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java index d00ebb82e..a1af003bb 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationAppsTest.java @@ -25,7 +25,6 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; -import java.util.Set; import java.util.stream.Collectors; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java index f68d64c3c..ea4e8cf0c 100644 --- a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/kafka/CdcKafkaReplicationTest.java @@ -93,7 +93,7 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActivePassiveCdc(String cache) { - return startActivePassiveCdcWithFilters(cache, null, null); + return startActivePassiveCdcWithFilters(cache, "", ""); } /** {@inheritDoc} */ @@ -138,7 +138,7 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActiveActiveCdc() { - return startActiveActiveCdcWithFilters(null, null); + return startActiveActiveCdcWithFilters("", ""); } /** {@inheritDoc} */ @@ -275,8 +275,8 @@ private void checkK2IMetrics(Function longMetric) { * @param topic Kafka topic name. * @param metadataTopic Metadata topic name. * @param cache Cache name to stream to kafka. - * @param includeTemplate Include regex templates for cache names. - * @param excludeTemplate Exclude regex templates for cache names. + * @param includeTemplate Include regex template for cache names. + * @param excludeTemplate Exclude regex template for cache names. * @return Future for Change Data Capture application. */ protected IgniteInternalFuture igniteToKafka( @@ -318,8 +318,8 @@ protected IgniteInternalFuture igniteToKafka( * @param cacheName Cache name. * @param igniteCfg Ignite configuration. * @param dest Destination Ignite cluster. - * @param includeTemplate Include regex templates for cache names. - * @param excludeTemplate Exclude regex templates for cache names. + * @param includeTemplate Include regex template for cache names. + * @param excludeTemplate Exclude regex template for cache names. * @return Future for runed {@link KafkaToIgniteCdcStreamer}. */ protected IgniteInternalFuture kafkaToIgnite(