Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
26 commits
Select commit Hold shift + click to select a range
6eb1c44
IGNITE-22530 CDC: Add regex filters for cache names
Aug 30, 2024
233ea74
IGNITE-22530 Make caches set in KafkaToIgniteCdcStreamerApplier mutable
Oct 11, 2024
46dfe92
IGNITE-22530 Add removal of destroyed caches from cacheList file
Oct 12, 2024
3068908
IGNITE-22530 Add atomic write to caches file
lordgarrish Nov 13, 2024
9d6e7d7
IGNITE-22530 Add CdcConsumerEx interface
Jun 4, 2025
853ac02
IGNITE-22530 Add CdcRegexManager
Jun 8, 2025
f4122ae
IGNITE-22530 Refactor AbstractIgniteCdcStreamer for use with CdcRegex…
Jun 8, 2025
a62cd36
IGNITE-22530 Refactor IgniteToKafkaCdcStreamer for use with CdcRegexM…
lordgarrish Jun 9, 2025
0a10b2a
IGNITE-22530 Add minor refactor
Jul 9, 2025
d59a1b3
IGNITE-22530 Add Javadoc
lordgarrish Aug 5, 2025
7d0bf3c
IGNITE-22530 Fix indentation
lordgarrish Aug 5, 2025
1192699
IGNITE-22530 Remove usage of CdcRegexMatcher interface
Nov 3, 2025
5ce5daf
IGNITE-22530 WIP
Nov 3, 2025
8d38ba2
IGNITE-22530 Make each regex pattern a single string
lordgarrish Nov 8, 2025
1593453
IGNITE-22530 Add already existing caches to CDC after new regex filte…
Jan 6, 2026
77e1797
IGNITE-22530 Refactor CdcRegexManager
Jan 30, 2026
c969dd9
IGNITE-22530 Add minor refactoring
lordgarrish Jan 31, 2026
04ee899
IGNITE-22530 Add new test
Feb 3, 2026
50f0e1c
IGNITE-22530 Minor fix
Feb 3, 2026
19f6ed0
IGNITE-22530 Add licenses
lordgarrish Feb 5, 2026
eb3d3c6
IGNITE-22530 Remove redundant regex filters from Kafka2Ignite
lordgarrish Feb 25, 2026
99b85e8
IGNITE-22530 Refactor CdcRegexManager
lordgarrish Apr 4, 2026
ebb84f1
IGNITE-22530 Refactor usage of CdcRegexManager
lordgarrish Apr 5, 2026
e014445
IGNITE-22530 Add new test
Apr 8, 2026
c71aca8
IGNITE-22530 Make cachesIds set in streamers mutable
lordgarrish Apr 9, 2026
22874d5
IGNITE-22530 Refactor CdcRegexManager
Apr 11, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,20 @@

package org.apache.ignite.cdc;

import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;

import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.binary.BinaryType;
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;
Expand All @@ -42,7 +46,7 @@
*
* @see AbstractCdcEventsApplier
*/
public abstract class AbstractIgniteCdcStreamer implements CdcConsumer {
public abstract class AbstractIgniteCdcStreamer implements CdcConsumerEx {
/** */
public static final String EVTS_SENT_CNT = "EventsCount";

Expand Down Expand Up @@ -73,6 +77,15 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer {
/** Cache names. */
private Set<String> caches;

/** Regexp manager. */
private CdcRegexManager regexManager;

/** Include regex template for cache names. */
private String includeTemplate;

/** Exclude regex template for cache names. */
private String excludeTemplate;

/** Cache IDs. */
protected Set<Integer> cachesIds;

Expand Down Expand Up @@ -100,12 +113,26 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer {

/** {@inheritDoc} */
@Override public void start(MetricRegistry reg) {
//No-op
}

/** {@inheritDoc} */
@Override public void start(MetricRegistry reg, List<String> cacheNames) {
A.notEmpty(caches, "caches");

regexManager = new CdcRegexManager(log);

cachesIds = caches.stream()
.mapToInt(CU::cacheId)
.boxed()
.collect(Collectors.toSet());
.collect(Collectors.toCollection(HashSet::new));

regexManager.compileRegexp(includeTemplate, excludeTemplate);

cacheNames.stream()
.filter(regexManager::matchesFilters)
.map(CU::cacheId)
.forEach(cachesIds::add);

MetricRegistryImpl mreg = (MetricRegistryImpl)reg;

Expand Down Expand Up @@ -144,15 +171,26 @@ public abstract class AbstractIgniteCdcStreamer implements CdcConsumer {
/** {@inheritDoc} */
@Override public void onCacheChange(Iterator<CdcCacheEvent> cacheEvents) {
cacheEvents.forEachRemaining(e -> {
// Just skip. Handle of cache events not supported.
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.
*
* @param cacheName Cache name.
*/
private void matchWithRegex(String cacheName) {
int cacheId = CU.cacheId(cacheName);

if (!cachesIds.contains(cacheId) && regexManager.matchesFilters(cacheName))
cachesIds.add(cacheId);
}

/** {@inheritDoc} */
@Override public void onCacheDestroy(Iterator<Integer> caches) {
caches.forEachRemaining(e -> {
// Just skip. Handle of cache events not supported.
});
caches.forEachRemaining(cachesIds::remove);
}

/** {@inheritDoc} */
Expand Down Expand Up @@ -238,6 +276,30 @@ public AbstractIgniteCdcStreamer setCaches(Set<String> caches) {
return this;
}

/**
* Sets include regex pattern that participates in CDC.
*
* @param includeTemplate Include regex template
* @return {@code this} for chaining.
*/
public AbstractIgniteCdcStreamer setIncludeCacheTemplate(String includeTemplate) {
this.includeTemplate = includeTemplate;

return this;
}

/**
* Sets exclude regex pattern that participates in CDC.
*
* @param excludeTemplate Exclude regex template
* @return {@code this} for chaining.
*/
public AbstractIgniteCdcStreamer setExcludeCacheTemplate(String excludeTemplate) {
this.excludeTemplate = excludeTemplate;

return this;
}

/**
* Sets maximum batch size that will be applied to destination cluster.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -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);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.ignite.cdc;

import java.util.List;

import org.apache.ignite.IgniteException;
import org.apache.ignite.Ignition;
import org.apache.ignite.cdc.conflictresolve.CacheVersionConflictResolverImpl;
Expand Down Expand Up @@ -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, List<String> cacheNames) {
super.start(mreg, cacheNames);

if (log.isInfoEnabled())
log.info("Ignite To Ignite Streamer [cacheIds=" + cachesIds + ']');
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.CdcRegexManager;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.internal.IgniteEx;
import org.apache.ignite.internal.processors.cache.version.CacheVersionConflictResolver;
Expand Down Expand Up @@ -65,6 +67,15 @@ public class CacheVersionConflictResolverPluginProvider<C extends PluginConfigur
/** Custom conflict resolver. */
private CacheVersionConflictResolver resolver;

/** Regexp manager. */
private CdcRegexManager regexManager;

/** Include regex templates for cache names. */
private String includeTemplate;

/** Exclude regex templates for cache names. */
private String excludeTemplate;

/** Log. */
private IgniteLogger log;

Expand Down Expand Up @@ -92,13 +103,15 @@ public CacheVersionConflictResolverPluginProvider() {
@Override public void initExtensions(PluginContext ctx, ExtensionRegistry registry) {
this.log = ctx.log(CacheVersionConflictResolverPluginProvider.class);
this.provider = new CacheVersionConflictResolverCachePluginProvider<>(conflictResolveField, clusterId, resolver);
this.regexManager = new CdcRegexManager(log);
regexManager.compileRegexp(includeTemplate, excludeTemplate);
}

/** {@inheritDoc} */
@Override public CachePluginProvider createCacheProvider(CachePluginContext ctx) {
String cacheName = ctx.igniteCacheConfiguration().getName();

if (caches.contains(cacheName)) {
if (caches.contains(cacheName) || regexManager.matchesFilters(cacheName)) {
log.info("ConflictResolver provider set for cache [cacheName=" + cacheName + ']');

return provider;
Expand Down Expand Up @@ -144,6 +157,16 @@ public void setConflictResolver(CacheVersionConflictResolver resolver) {
this.resolver = resolver;
}

/** @param includeTemplate Include regex template */
public void setIncludeCacheTemplate(String includeTemplate) {
this.includeTemplate = includeTemplate;
}

/** @param excludeTemplate Exclude regex template */
public void setExcludeCacheTemplate(String excludeTemplate) {
this.excludeTemplate = excludeTemplate;
}

/** {@inheritDoc} */
@Override public void start(PluginContext ctx) {
((IgniteEx)ctx.grid()).context().cache().context().versions().dataCenterId(clusterId);
Expand Down
Loading