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..3ef92ca06 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 @@ -19,7 +19,7 @@ import java.util.Iterator; import java.util.Set; -import java.util.stream.Collectors; + import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -27,11 +27,11 @@ 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; import org.apache.ignite.internal.util.typedef.internal.A; -import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.metric.MetricRegistry; import org.apache.ignite.resources.LoggerResource; @@ -42,7 +42,7 @@ * * @see AbstractCdcEventsApplier */ -public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { +public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx { /** */ public static final String EVTS_SENT_CNT = "EventsCount"; @@ -73,8 +73,8 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { /** Cache names. */ private Set caches; - /** Cache IDs. */ - protected Set cachesIds; + /** Caches predicate. */ + protected CachesPredicate cachesPredicate = new CachesPredicate(); /** Maximum batch size. */ protected int maxBatchSize; @@ -100,12 +100,18 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { /** {@inheritDoc} */ @Override public void start(MetricRegistry reg) { + //No-op + } + + /** {@inheritDoc} */ + @Override public void start(MetricRegistry reg, Iterator cacheEvents) { A.notEmpty(caches, "caches"); - cachesIds = caches.stream() - .mapToInt(CU::cacheId) - .boxed() - .collect(Collectors.toSet()); + cachesPredicate.setCaches(caches); + + cacheEvents.forEachRemaining(evt -> { + cachesPredicate.onCacheEvent(evt.configuration().getName()); + }); MetricRegistryImpl mreg = (MetricRegistryImpl)reg; @@ -123,7 +129,7 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { F.identity(), true, evt -> !onlyPrimary || evt.primary(), - evt -> F.isEmpty(cachesIds) || cachesIds.contains(evt.cacheId()), + evt -> cachesPredicate.test(evt.cacheId()), evt -> evt.version().otherClusterVersion() == null)); if (msgsSnt > 0) { @@ -144,15 +150,13 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer { /** {@inheritDoc} */ @Override public void onCacheChange(Iterator cacheEvents) { cacheEvents.forEachRemaining(e -> { - // Just skip. Handle of cache events not supported. + cachesPredicate.onCacheEvent(e.configuration().getName()); }); } /** {@inheritDoc} */ @Override public void onCacheDestroy(Iterator caches) { - caches.forEachRemaining(e -> { - // Just skip. Handle of cache events not supported. - }); + caches.forEachRemaining(cachesPredicate::onCacheDestroy); } /** {@inheritDoc} */ @@ -238,6 +242,30 @@ public AbstractIgniteCdcStreamer setCaches(Set caches) { return this; } + /** + * Sets include regex pattern for caches participating in CDC. + * + * @param includeRegex Include regex string + * @return {@code this} for chaining. + */ + public AbstractIgniteCdcStreamer setIncludeCachesRegex(String includeRegex) { + cachesPredicate.setIncludeCacheTemplate(includeRegex); + + return this; + } + + /** + * Sets exclude regex pattern for caches participating in CDC. + * + * @param excludeRegex Exclude regex string + * @return {@code this} for chaining. + */ + public AbstractIgniteCdcStreamer setExcludeCachesRegex(String excludeRegex) { + cachesPredicate.setExcludeCacheTemplate(excludeRegex); + + 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/CachesPredicate.java b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CachesPredicate.java new file mode 100644 index 000000000..20508665b --- /dev/null +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CachesPredicate.java @@ -0,0 +1,134 @@ +/* + * 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.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.function.Predicate; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; +import java.util.stream.Collectors; + +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.util.typedef.internal.CU; + +/** + * Predicate for filtering {@link CdcEvent}s inside {@link CdcConsumer#onEvents(Iterator)}. Filters out events for + * following types of caches: + *
    + *
  1. Caches set in CDC configuration.
  2. + *
  3. Caches that are added dynamically by user's cache regexp templates.
  4. + *
+ */ +public class CachesPredicate implements Predicate { + /** Include regex pattern for cache names. */ + private Pattern includePtrn; + + /** Exclude regex pattern for cache names. */ + private Pattern excludePtrn; + + /** Cache IDs. */ + private Set cacheIds; + + /** Cache regex IDs. */ + private final Set cacheRegexIds = new ConcurrentSkipListSet<>(); + + /** + * Sets cache ids of caches participating in CDC. + * @param caches Cache names. + */ + public void setCaches(Collection caches) { + cacheIds = caches.stream() + .mapToInt(CU::cacheId) + .boxed() + .collect(Collectors.toCollection(HashSet::new)); + } + + /** + * Sets include regex pattern for caches participating in CDC. + * + * @param includeRegex Include regex string + * @throws IgniteException If the template's syntax is invalid + */ + public void setIncludeCacheTemplate(String includeRegex) { + try { + includePtrn = includeRegex != null ? Pattern.compile(includeRegex) : Pattern.compile(""); + } + catch (PatternSyntaxException e) { + throw new IgniteException("Invalid cache regexp template", e); + } + } + + /** + * Sets exclude regex pattern for caches participating in CDC. + * + * @param excludeRegex Exclude regex string + * @throws IgniteException If the template's syntax is invalid + */ + public void setExcludeCacheTemplate(String excludeRegex) { + try { + excludePtrn = excludeRegex != null ? Pattern.compile(excludeRegex) : Pattern.compile(""); + } + catch (PatternSyntaxException e) { + throw new IgniteException("Invalid cache regexp template", e); + } + } + + /** {@inheritDoc} */ + @Override public boolean test(Integer cacheId) { + return cacheIds.contains(cacheId) || cacheRegexIds.contains(cacheId); + } + + /** + * 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. + */ + public boolean onCacheEvent(String cacheName) { + if (excludePtrn.matcher(cacheName).matches()) + return false; + + if (includePtrn.matcher(cacheName).matches()) + cacheRegexIds.add(CU.cacheId(cacheName)); + + return true; + } + + /** + * Removes destroyed cache from replication. + * @param cacheId Cache id. + * */ + public void onCacheDestroy(int cacheId) { + cacheRegexIds.remove(cacheId); + } + + /** + * @return {@link Set} of cache ids participating in CDC. + */ + public Set getCacheIds() { + Set cacheIds = new HashSet<>(this.cacheIds) ; + + cacheIds.addAll(cacheRegexIds); + + return cacheIds; + } +} 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..76948c28d --- /dev/null +++ b/modules/cdc-ext/src/main/java/org/apache/ignite/cdc/CdcRegexManager.java @@ -0,0 +1,83 @@ +/* + * 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.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; + +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; + +/** + * Contains logic to process user's regexp patterns for CDC. + */ +public class CdcRegexManager { + + /** Include regex pattern for cache names. */ + private Pattern includeFilter; + + /** Exclude regex pattern for cache names. */ + private Pattern excludeFilter; + + /** Logger. */ + private IgniteLogger log; + + /** + * + * @param log Logger. + */ + public CdcRegexManager(IgniteLogger log) { + this.log = log; + } + + /** + * 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. + */ + public boolean matchesFilters(String cacheName) { + if (includeFilter.matcher(cacheName).matches() && excludeFilter.matcher(cacheName).matches()) { + if (log.isInfoEnabled()) + log.info("Cache name matches both include and exclude regexp templates. Will except this cache from " + + "replication [cacheName=" + cacheName + ", includeTemplate=" + includeFilter + ", excludeTemplate=" + + excludeFilter + "]"); + + return false; + } + return includeFilter.matcher(cacheName).matches(); + } + + /** + * Compiles regex patterns from user templates. + * + * @param includeTemplate Include regex template. + * @param excludeTemplate Exclude regex template. + * @throws IgniteException If the template's syntax is invalid + */ + public void compileRegexp(String includeTemplate, String excludeTemplate) { + try { + includeFilter = includeTemplate != null ? Pattern.compile(includeTemplate) : Pattern.compile(""); + + excludeFilter = excludeTemplate != null ? Pattern.compile(excludeTemplate) : Pattern.compile(""); + } + catch (PatternSyntaxException e) { + throw new IgniteException("Invalid cache regexp template", e); + } + } +} 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..d533eb4b0 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.util.Iterator; + import org.apache.ignite.IgniteException; import org.apache.ignite.Ignition; import org.apache.ignite.cdc.conflictresolve.CacheVersionConflictResolverImpl; @@ -59,11 +61,11 @@ 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, Iterator cacheEvents) { + super.start(mreg, cacheEvents); if (log.isInfoEnabled()) - log.info("Ignite To Ignite Streamer [cacheIds=" + cachesIds + ']'); + log.info("Ignite To Ignite Streamer [cacheIds=" + cachesPredicate.getCacheIds() + ']'); A.notNull(destIgniteCfg, "Destination Ignite configuration."); 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..acbe74d6a 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 @@ -20,7 +20,9 @@ import java.io.Serializable; import java.util.Set; import java.util.UUID; + import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cdc.CachesPredicate; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cache.version.CacheVersionConflictResolver; @@ -65,6 +67,9 @@ public class CacheVersionConflictResolverPluginProvider cachesIds; - /** Cache names. */ private Collection caches; + /** Caches predicate. */ + protected CachesPredicate cachesPredicate = new CachesPredicate(); + /** Max batch size. */ private int maxBatchSz = DFLT_MAX_BATCH_SIZE; @@ -197,7 +196,7 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumer { return false; } - if (!cachesIds.contains(evt.cacheId())) { + if (!cachesPredicate.test(evt.cacheId())) { if (log.isDebugEnabled()) log.debug("Event skipped because of cacheId [evt=" + evt + ']'); @@ -246,15 +245,13 @@ public class IgniteToKafkaCdcStreamer implements CdcConsumer { /** {@inheritDoc} */ @Override public void onCacheChange(Iterator cacheEvents) { cacheEvents.forEachRemaining(e -> { - // Just skip. Handle of cache events not supported. + cachesPredicate.onCacheEvent(e.configuration().getName()); }); } /** {@inheritDoc} */ @Override public void onCacheDestroy(Iterator caches) { - caches.forEachRemaining(e -> { - // Just skip. Handle of cache events not supported. - }); + caches.forEachRemaining(cachesPredicate::onCacheDestroy); } /** Send marker(meta need to be updated) record to each partition of events topic. */ @@ -319,6 +316,11 @@ private void sendOneBatch( /** {@inheritDoc} */ @Override public void start(MetricRegistry reg) { + //No-op + } + + /** {@inheritDoc} */ + @Override public void start(MetricRegistry reg, Iterator cacheEvents) { A.notNull(kafkaProps, "Kafka properties"); A.notNull(evtTopic, "Kafka topic"); A.notNull(metadataTopic, "Kafka metadata topic"); @@ -329,9 +331,11 @@ private void sendOneBatch( kafkaProps.setProperty(KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class.getName()); kafkaProps.setProperty(VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); - cachesIds = caches.stream() - .map(CU::cacheId) - .collect(Collectors.toSet()); + cachesPredicate.setCaches(caches); + + cacheEvents.forEachRemaining(evt -> { + cachesPredicate.onCacheEvent(evt.configuration().getName()); + }); try { producer = new KafkaProducer<>(kafkaProps); @@ -341,7 +345,7 @@ private void sendOneBatch( "topic=" + evtTopic + ", metadataTopic = " + metadataTopic + ", onlyPrimary=" + onlyPrimary + - ", cacheIds=" + cachesIds + ']' + ", cacheIds=" + cachesPredicate.getCacheIds() + ']' ); } } @@ -426,6 +430,30 @@ public IgniteToKafkaCdcStreamer setCaches(Collection caches) { return this; } + /** + * Sets include regex pattern that participates in CDC. + * + * @param includeRegex Include regex template. + * @return {@code this} for chaining. + */ + public IgniteToKafkaCdcStreamer setIncludeCachesRegex(String includeRegex) { + cachesPredicate.setIncludeCacheTemplate(includeRegex); + + return this; + } + + /** + * Sets exclude regex pattern that participates in CDC. + * + * @param excludeRegex Exclude regex template. + * @return {@code this} for chaining. + */ + public IgniteToKafkaCdcStreamer setExcludeCachesRegex(String excludeRegex) { + cachesPredicate.setExcludeCacheTemplate(excludeRegex); + + return this; + } + /** * Sets maximum batch size. * 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..cbdcf6d97 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,8 +17,11 @@ package org.apache.ignite.cdc.thin; +import java.util.Iterator; + import org.apache.ignite.Ignition; import org.apache.ignite.cdc.AbstractIgniteCdcStreamer; +import org.apache.ignite.cdc.CdcCacheEvent; import org.apache.ignite.cdc.conflictresolve.CacheVersionConflictResolverImpl; import org.apache.ignite.cdc.kafka.KafkaToIgniteCdcStreamer; import org.apache.ignite.client.ClientException; @@ -66,11 +69,11 @@ 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, Iterator cacheEvents) { + super.start(mreg, cacheEvents); if (log.isInfoEnabled()) - log.info("Ignite To Ignite Client Streamer [cacheIds=" + cachesIds + ']'); + log.info("Ignite To Ignite Client Streamer [cacheIds=" + cachesPredicate.getCacheIds() + ']'); A.notNull(destClientCfg, "Destination thin client configuration"); 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..0c7abd0cb 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 @@ -146,6 +146,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 +212,8 @@ private enum WaitDataMode { cfgPlugin1.setClusterId(clusterId); cfgPlugin1.setCaches(new HashSet<>(Arrays.asList(ACTIVE_PASSIVE_CACHE, ACTIVE_ACTIVE_CACHE))); + cfgPlugin1.setIncludeCachesRegex(REGEX_INCLUDE_PATTERN); + cfgPlugin1.setExcludeCachesRegex(REGEX_EXCLUDE_PATTERN); cfgPlugin1.setConflictResolveField("reqId"); cfg.setPluginProviders(cfgPlugin1); @@ -562,6 +576,92 @@ 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 { + 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(REGEX_INCLUDE_PATTERN, REGEX_EXCLUDE_PATTERN); + + 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 { + //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); + + 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 +788,18 @@ protected String[] hostAddresses(IgniteEx[] dest) { /** */ protected abstract List> startActivePassiveCdc(String cache); + /** */ + protected abstract List> startActivePassiveCdcWithFilters(String cache, + String includeTemplate, + String excludeTemplate); + /** */ protected abstract List> startActiveActiveCdc(); + /** */ + 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 b6d42e240..98872065d 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 @@ -45,26 +45,40 @@ public class CdcIgniteToIgniteReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActivePassiveCdc(String cache) { + return startActivePassiveCdcWithFilters(cache, "", ""); + } + + /** {@inheritDoc} */ + @Override protected List> startActivePassiveCdcWithFilters(String cache, + 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, "ignite-to-ignite-src-" + i)); + futs.add(igniteToIgnite(srcCluster[i].configuration(), destClusterCliCfg[i], destCluster, cache, + includeTemplate, excludeTemplate, "ignite-to-ignite-src-" + i)); return futs; } /** {@inheritDoc} */ @Override protected List> startActiveActiveCdc() { + return startActiveActiveCdcWithFilters("", ""); + } + + /** {@inheritDoc} */ + @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, "ignite-to-ignite-src-" + i)); + futs.add(igniteToIgnite(srcCluster[i].configuration(), destClusterCliCfg[i], destCluster, + 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, "ignite-to-ignite-dest-" + i)); + futs.add(igniteToIgnite(destCluster[i].configuration(), srcClusterCliCfg[i], srcCluster, + ACTIVE_ACTIVE_CACHE, includeTemplate, excludeTemplate, "ignite-to-ignite-dest-" + i)); } return futs; @@ -86,6 +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 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. */ @@ -94,6 +110,8 @@ protected IgniteInternalFuture igniteToIgnite( IgniteConfiguration destCfg, IgniteEx[] dest, String cache, + String includeTemplate, + String excludeTemplate, @Nullable String threadName ) { return runAsync(() -> { @@ -115,6 +133,8 @@ protected IgniteInternalFuture igniteToIgnite( streamer.setMaxBatchSize(KEYS_CNT); streamer.setCaches(Collections.singleton(cache)); + streamer.setIncludeCachesRegex(includeTemplate); + streamer.setExcludeCachesRegex(excludeTemplate); cdcCfg.setConsumer(streamer); cdcCfg.setMetricExporterSpi(new JmxMetricExporterSpi()); diff --git a/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersSelfTest.java b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersSelfTest.java new file mode 100644 index 000000000..07a909018 --- /dev/null +++ b/modules/cdc-ext/src/test/java/org/apache/ignite/cdc/RegexFiltersSelfTest.java @@ -0,0 +1,399 @@ +/* + * 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.util.Collections; +import java.util.Set; +import java.util.stream.IntStream; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +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.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +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 RegexFiltersSelfTest extends GridCommonAbstractTest { + + /** */ + private IgniteEx src; + + /** */ + private IgniteEx dest; + + /** */ + private AbstractIgniteCdcStreamer streamer; + + /** */ + 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_1 = "regex-cache1"; + + /** */ + private static final String REGEX_MATCHING_CACHE_2 = "regex-cache2"; + + /** */ + private static final String REGEX_MATCHING_CACHE_3 = "my-cache1"; + + /** */ + private static final String REGEX_EXCLUDED_CACHE = "forbidden-cache1"; + + /** */ + private static final String REGEX_INCLUDE_PATTERN = "regex.*"; + + /** */ + private static final String REGEX_EXCLUDE_PATTERN = "forbidden.*"; + + /** */ + private static final String INVALID_PATTERN = "[aaaaaaaaaaaaa"; + + /** */ + 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.setCacheConfiguration(new CacheConfiguration<>() + .setName(REGEX_MATCHING_CACHE_2) + .setCacheMode(PARTITIONED) + .setAtomicityMode(TRANSACTIONAL)); + + cfg.setConsistentId(igniteInstanceName); + + return cfg; + } + + /** + * + * @param srcCfg Ignite source node configuration. + * @param cache Cache name to stream to Ignite2Ignite. + * @param includeTemplate Include cache template. + * @param excludeTemplate Exclude cache template. + * @return Future for Change Data Capture application. + */ + private IgniteInternalFuture startCdc( + IgniteConfiguration srcCfg, + String cache, + String includeTemplate, + String excludeTemplate + ) { + return runAsync(() -> { + CdcConfiguration cdcCfg = new CdcConfiguration(); + + streamer = new TestI2IClientCdcStreamer() + .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.setIncludeCachesRegex(includeTemplate); + streamer.setExcludeCachesRegex(excludeTemplate); + + 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")); + } + + /** {@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)); + }; + } + + /** */ + public IgniteCache getCache(IgniteEx cluster, String cacheName) { + return cluster.getOrCreateCache(new CacheConfiguration() + .setName(cacheName) + .setCacheMode(PARTITIONED) + .setAtomicityMode(TRANSACTIONAL)); + } + + /** + * Test checks whether caches added by regex filters are returned to replication 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' set in 'caches' property of streamer config and cache masks (regex filters) + IgniteInternalFuture cdc = startCdc(src.configuration(), TEST_CACHE, REGEX_INCLUDE_PATTERN, ""); + + IgniteCache srcCache = getCache(src, REGEX_MATCHING_CACHE_1); + + IgniteCache destCache = getCache(dest, REGEX_MATCHING_CACHE_1); + + IgniteCache srcCache2 = src.getOrCreateCache(REGEX_MATCHING_CACHE_2); + + IgniteCache destCache2 = dest.getOrCreateCache(REGEX_MATCHING_CACHE_2); + + //Cache created through CacheConfiguration on cluster start should be also added to CDC + try { + runAsync(generateData(srcCache2, IntStream.range(0, KEYS_CNT))); + + waitForSameData(srcCache2, destCache2, KEYS_CNT, WaitDataMode.EXISTS, cdc); + } + finally { + cdc.cancel(); + } + + //Restart CDC + IgniteInternalFuture cdc2 = startCdc(src.configuration(), TEST_CACHE, REGEX_INCLUDE_PATTERN, ""); + + try { + runAsync(generateData(srcCache, IntStream.range(0, KEYS_CNT))); + + waitForSameData(srcCache, destCache, KEYS_CNT, WaitDataMode.EXISTS, cdc2); + } + finally { + cdc2.cancel(); + } + } + + /** + * Test checks whether after changing regex filters config, existing caches matching new criteria are added to CDC. + */ + @Test + public void testRegexConfigChange() throws Exception { + src.cluster().state(ClusterState.ACTIVE); + + dest.cluster().state(ClusterState.ACTIVE); + + IgniteInternalFuture cdc = startCdc(src.configuration(), TEST_CACHE, REGEX_INCLUDE_PATTERN, ""); + + IgniteCache srcCache = getCache(src, REGEX_MATCHING_CACHE_1); + + IgniteCache destCache = getCache(dest, REGEX_MATCHING_CACHE_1); + + IgniteCache srcCache2 = getCache(src, REGEX_MATCHING_CACHE_3); + + IgniteCache destCache2 = getCache(dest, REGEX_MATCHING_CACHE_3); + + cdc.cancel(); + + final String NEW_REGEX_PATTERN = REGEX_INCLUDE_PATTERN + "|my-cache.*"; + + //Restart CDC with the new regexp config + IgniteInternalFuture cdc2 = startCdc(src.configuration(), TEST_CACHE, NEW_REGEX_PATTERN, ""); + + try { + runAsync(generateData(srcCache, IntStream.range(0, KEYS_CNT))); + + runAsync(generateData(srcCache2, IntStream.range(0, KEYS_CNT))); + + waitForSameData(srcCache, destCache, KEYS_CNT, WaitDataMode.EXISTS, cdc2); + + waitForSameData(srcCache2, destCache2, KEYS_CNT, WaitDataMode.EXISTS, cdc2); + } + finally { + cdc2.cancel(); + } + } + + /** + * Test checks whether caches matching exclude regex filters are excluded from CDC. + */ + @Test + public void testRegexExcludeFilters() throws Exception { + src.cluster().state(ClusterState.ACTIVE); + + dest.cluster().state(ClusterState.ACTIVE); + + doTestCdcExclude(src.configuration(), TEST_CACHE, "", REGEX_EXCLUDE_PATTERN, REGEX_EXCLUDED_CACHE); + } + + /** + * Test checks whether caches matching both regex filters are excluded from CDC. + */ + @Test + public void testCacheNameMatchesBothPatterns() throws Exception { + src.cluster().state(ClusterState.ACTIVE); + + dest.cluster().state(ClusterState.ACTIVE); + + doTestCdcExclude(src.configuration(), TEST_CACHE, REGEX_INCLUDE_PATTERN, REGEX_INCLUDE_PATTERN, REGEX_MATCHING_CACHE_1); + } + + /** */ + private void doTestCdcExclude( + IgniteConfiguration srcCfg, + String cache, + String includeTemplate, + String excludeTemplate, + String excludedCache + ) throws Exception { + IgniteInternalFuture cdc = startCdc(srcCfg, cache, includeTemplate, excludeTemplate); + + IgniteCache srcCache = getCache(src, excludedCache); + + runAsync(generateData(srcCache, IntStream.range(0, KEYS_CNT))); + + assertTrue(waitForCondition(() -> srcCache.size() == KEYS_CNT, getTestTimeout())); + + TestI2IClientCdcStreamer strmr = (TestI2IClientCdcStreamer)streamer; + + assertEquals(1, strmr.getCacheIds().size()); + + assertTrue(strmr.getCacheIds().contains(TEST_CACHE.hashCode())); + + cdc.cancel(); + } + + /** + * Test checks that CDC won't start if invalid cache regexp is set. + */ + @Test + public void testInvalidRegex() throws Exception { + src.cluster().state(ClusterState.ACTIVE); + + dest.cluster().state(ClusterState.ACTIVE); + + IgniteInternalFuture cdc = startCdc(src.configuration(), TEST_CACHE, INVALID_PATTERN, ""); + + waitForCondition(() -> cdc.error() != null, getTestTimeout()); + + assertEquals(IgniteException.class, cdc.error().getClass()); + + assertEquals("Invalid cache regexp template", cdc.error().getMessage()); + + cdc.cancel(); + } + + /** */ + private static class TestI2IClientCdcStreamer extends IgniteToIgniteClientCdcStreamer { + /** */ + public Set getCacheIds() { + return cachesPredicate.getCacheIds(); + } + } +} 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..4d3cb987b 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 @@ -113,6 +113,8 @@ public class CdcKafkaReplicationAppsTest extends CdcKafkaReplicationTest { String topic, String metadataTopic, String cache, + String includeTemplate, + String excludeTemplate, String threadName ) { Map params = new HashMap<>(); @@ -141,6 +143,7 @@ public class CdcKafkaReplicationAppsTest extends CdcKafkaReplicationTest { IgniteEx[] dest, int partFrom, int partTo, + boolean withRegexp, String threadName ) { Map params = new HashMap<>(); @@ -165,7 +168,9 @@ public class CdcKafkaReplicationAppsTest extends CdcKafkaReplicationTest { params.put(DISCO_PORT_RANGE, Integer.toString(discoPort + DFLT_PORT_RANGE)); } - params.put(REPLICATED_CACHE, cacheName); + if (!withRegexp) + params.put(REPLICATED_CACHE, cacheName); + params.put(TOPIC, topic); params.put(METADATA_TOPIC, metadataTopic); params.put(PROPS_PATH, kafkaPropsPath); 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..ca6d0ff1e 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,6 +93,13 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActivePassiveCdc(String cache) { + return startActivePassiveCdcWithFilters(cache, "", ""); + } + + /** {@inheritDoc} */ + @Override protected List> startActivePassiveCdcWithFilters(String cache, + String includeTemplate, + String excludeTemplate) { try { KAFKA.createTopic(cache, DFLT_PARTS, 1); @@ -107,9 +114,12 @@ 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, includeTemplate, + excludeTemplate, "ignite-src-to-kafka-" + idx)); } + boolean withRegexp = !includeTemplate.isEmpty() || !excludeTemplate.isEmpty(); + for (int i = 0; i < destCluster.length; i++) { futs.add(kafkaToIgnite( cache, @@ -119,7 +129,8 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { destCluster, i * (DFLT_PARTS / 2), (i + 1) * (DFLT_PARTS / 2), - "kafka-to-ignite-dest-" + i + withRegexp, + "kafka-to-ignite-dest-" + i )); } @@ -128,22 +139,30 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { /** {@inheritDoc} */ @Override protected List> startActiveActiveCdc() { + return startActiveActiveCdcWithFilters("", ""); + } + + /** {@inheritDoc} */ + @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, "ignite-src-to-kafka-" + idx)); + + futs.add(igniteToKafka(ex.configuration(), SRC_DEST_TOPIC, SRC_DEST_META_TOPIC, ACTIVE_ACTIVE_CACHE, + 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, "ignite-dest-to-kafka-" + idx)); + + futs.add(igniteToKafka(ex.configuration(), DEST_SRC_TOPIC, DEST_SRC_META_TOPIC, ACTIVE_ACTIVE_CACHE, + includeTemplate, excludeTemplate, "ignite-dest-to-kafka-" + idx)); } + boolean withRegexp = !includeTemplate.isEmpty() || !excludeTemplate.isEmpty(); + futs.add(kafkaToIgnite( ACTIVE_ACTIVE_CACHE, SRC_DEST_TOPIC, @@ -152,6 +171,7 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { destCluster, 0, DFLT_PARTS, + withRegexp, "kafka-to-ignite-src" )); @@ -163,6 +183,7 @@ public class CdcKafkaReplicationTest extends AbstractReplicationTest { srcCluster, 0, DFLT_PARTS, + withRegexp, "kafka-to-ignite-dest" )); @@ -255,6 +276,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 template for cache names. + * @param excludeTemplate Exclude regex template for cache names. * @return Future for Change Data Capture application. */ protected IgniteInternalFuture igniteToKafka( @@ -262,6 +285,8 @@ protected IgniteInternalFuture igniteToKafka( String topic, String metadataTopic, String cache, + String includeTemplate, + String excludeTemplate, String threadName ) { return runAsync(() -> { @@ -270,6 +295,8 @@ protected IgniteInternalFuture igniteToKafka( .setMetadataTopic(metadataTopic) .setKafkaPartitions(DFLT_PARTS) .setCaches(Collections.singleton(cache)) + .setIncludeCachesRegex(includeTemplate) + .setExcludeCachesRegex(excludeTemplate) .setMaxBatchSize(KEYS_CNT) .setOnlyPrimary(false) .setKafkaProperties(kafkaProperties()) @@ -292,6 +319,7 @@ protected IgniteInternalFuture igniteToKafka( * @param cacheName Cache name. * @param igniteCfg Ignite configuration. * @param dest Destination Ignite cluster. + * @param withRegexp Flag indicating whether regexp filters for CDC are set * @return Future for runed {@link KafkaToIgniteCdcStreamer}. */ protected IgniteInternalFuture kafkaToIgnite( @@ -302,6 +330,7 @@ protected IgniteInternalFuture kafkaToIgnite( IgniteEx[] dest, int fromPart, int toPart, + boolean withRegexp, String threadName ) { KafkaToIgniteCdcStreamerConfiguration cfg = new KafkaToIgniteCdcStreamerConfiguration(); @@ -310,7 +339,11 @@ protected IgniteInternalFuture kafkaToIgnite( cfg.setKafkaPartsTo(toPart); cfg.setThreadCount((toPart - fromPart) / 2); - cfg.setCaches(Collections.singletonList(cacheName)); + if (withRegexp) + cfg.setCaches(Collections.emptyList()); + else + cfg.setCaches(Collections.singletonList(cacheName)); + cfg.setTopic(topic); cfg.setMetadataTopic(metadataTopic); cfg.setKafkaRequestTimeout(DFLT_KAFKA_REQ_TIMEOUT);