anton-vinogradov commented on a change in pull request #49:
URL: https://github.com/apache/ignite-extensions/pull/49#discussion_r647398276



##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/KafkaUtils.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
+
+import static org.apache.ignite.cdc.Utils.property;
+
+/** Kafka Utils. */
+public class KafkaUtils {
+    /** Default kafka request timeout. */
+    public static final int TIMEOUT_MIN = 1;
+
+    /** Ignite to Kafka topic partitions number. */
+    public static final String IGNITE_TO_KAFKA_NUM_PARTITIONS = 
"ignite.to.kafka.numpartitions";

Review comment:
       partitions.amount?
   
   BTW, proper place for this property is a `ChangeDataCaptureIgniteToKafka`

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/KafkaUtils.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
+
+import static org.apache.ignite.cdc.Utils.property;
+
+/** Kafka Utils. */
+public class KafkaUtils {
+    /** Default kafka request timeout. */
+    public static final int TIMEOUT_MIN = 1;
+
+    /** Ignite to Kafka topic partitions number. */
+    public static final String IGNITE_TO_KAFKA_NUM_PARTITIONS = 
"ignite.to.kafka.numpartitions";
+
+    /** Ignite to Kafka topic partitions number. */
+    public static final String IGNITE_TO_KAFKA_REPLICATION_FACTOR = 
"ignite.to.kafka.replication.factor";

Review comment:
       The proper place for this property is a `ChangeDataCaptureIgniteToKafka`
   and this means you can get rid of IGNITE_TO_KAFKA` prefix

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureKafkaToIgnite.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.ignite.cdc.conflictplugin.CacheConflictResolutionManagerImpl;
+import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverImpl;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.cdc.ChangeDataCapture;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.serialization.IntegerDeserializer;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.cdc.Utils.property;
+
+/**
+ * Main class of Kafka to Ignite application.
+ * This application is counterpart of {@link ChangeDataCaptureIgniteToKafka} 
Change Data Capture consumer.
+ * Application runs several {@link Applier} thread to read Kafka topic 
partitions and apply {@link ChangeDataCaptureEvent} to Ignite.
+ * <p>
+ * Each applier receive even number of kafka topic partition to read.
+ * <p>
+ * In case of any error during read applier just fail. Fail of any applier 
will lead to the fail of whole application.
+ * It expected that application will be configured for automatic restarts with 
the OS tool to failover temporary errors
+ * such as Kafka or Ignite unavailability.
+ * <p>
+ * To resolve possible update conflicts(in case of concurrent update in source 
and destination Ignite clusters)
+ * real-world deployments should use some conflict resolver, for example 
{@link CacheVersionConflictResolverImpl}.
+ * Example of Ignite configuration with the conflict resolver:
+ * <pre>
+ * {@code
+ * CacheVersionConflictResolverCachePluginProvider conflictPlugin = new 
CacheVersionConflictResolverCachePluginProvider();
+ *
+ * conflictPlugin.setClusterId(clusterId); // Cluster id.
+ * conflictPlugin.setCaches(new HashSet<>(Arrays.asList("my-cache", 
"some-other-cache"))); // Caches to replicate.
+ *
+ * IgniteConfiguration cfg = ...;
+ *
+ * cfg.setPluginProviders(conflictPlugin);
+ * }
+ * </pre>
+ * Please, see {@link CacheConflictResolutionManagerImpl} for additional 
information.
+ *
+ * Properties list:
+ * <ul>
+ *  <li>{@link #KAFKA_TO_IGNITE_THREAD_COUNT} - count of {@link Applier} 
threads.</li>
+ *  <li>{@link #IGNITE_TO_KAFKA_TOPIC} - Kafka topic name if not provided in 
constructor.</li>
+ * </ul>
+ *
+ * @see ChangeDataCapture
+ * @see ChangeDataCaptureIgniteToKafka
+ * @see ChangeDataCaptureEvent
+ * @see Applier
+ * @see CacheConflictResolutionManagerImpl
+ */
+public class ChangeDataCaptureKafkaToIgnite implements Runnable {

Review comment:
       KafkaChangeDataCaptureSink/Producer?

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureKafkaToIgnite.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.ignite.cdc.conflictplugin.CacheConflictResolutionManagerImpl;
+import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverImpl;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.cdc.ChangeDataCapture;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.serialization.IntegerDeserializer;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.cdc.Utils.property;
+
+/**
+ * Main class of Kafka to Ignite application.
+ * This application is counterpart of {@link ChangeDataCaptureIgniteToKafka} 
Change Data Capture consumer.
+ * Application runs several {@link Applier} thread to read Kafka topic 
partitions and apply {@link ChangeDataCaptureEvent} to Ignite.
+ * <p>
+ * Each applier receive even number of kafka topic partition to read.
+ * <p>
+ * In case of any error during read applier just fail. Fail of any applier 
will lead to the fail of whole application.
+ * It expected that application will be configured for automatic restarts with 
the OS tool to failover temporary errors
+ * such as Kafka or Ignite unavailability.
+ * <p>
+ * To resolve possible update conflicts(in case of concurrent update in source 
and destination Ignite clusters)
+ * real-world deployments should use some conflict resolver, for example 
{@link CacheVersionConflictResolverImpl}.
+ * Example of Ignite configuration with the conflict resolver:
+ * <pre>
+ * {@code
+ * CacheVersionConflictResolverCachePluginProvider conflictPlugin = new 
CacheVersionConflictResolverCachePluginProvider();
+ *
+ * conflictPlugin.setClusterId(clusterId); // Cluster id.
+ * conflictPlugin.setCaches(new HashSet<>(Arrays.asList("my-cache", 
"some-other-cache"))); // Caches to replicate.
+ *
+ * IgniteConfiguration cfg = ...;
+ *
+ * cfg.setPluginProviders(conflictPlugin);
+ * }
+ * </pre>
+ * Please, see {@link CacheConflictResolutionManagerImpl} for additional 
information.
+ *
+ * Properties list:
+ * <ul>
+ *  <li>{@link #KAFKA_TO_IGNITE_THREAD_COUNT} - count of {@link Applier} 
threads.</li>
+ *  <li>{@link #IGNITE_TO_KAFKA_TOPIC} - Kafka topic name if not provided in 
constructor.</li>
+ * </ul>
+ *
+ * @see ChangeDataCapture
+ * @see ChangeDataCaptureIgniteToKafka
+ * @see ChangeDataCaptureEvent
+ * @see Applier
+ * @see CacheConflictResolutionManagerImpl
+ */
+public class ChangeDataCaptureKafkaToIgnite implements Runnable {
+    /** Property to define number of {@link Applier} threads. */
+    private static final String KAFKA_TO_IGNITE_THREAD_COUNT = 
"kafka.to.ignite.thread.count";
+
+    /** Ignite to Kafka topic name. */
+    public static final String IGNITE_TO_KAFKA_TOPIC = "ignite.to.kafka.topic";
+
+    /** Ignite to Kafka maximum batch size. */
+    public static final String IGNITE_TO_KAFKA_MAX_BATCH_SZ = 
"ignite.to.kafka.max.batch.size";

Review comment:
       MAX_BATCH_SIZE
   
   why ignite TO kafka?

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/KafkaUtils.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
+
+import static org.apache.ignite.cdc.Utils.property;
+
+/** Kafka Utils. */
+public class KafkaUtils {
+    /** Default kafka request timeout. */
+    public static final int TIMEOUT_MIN = 1;

Review comment:
       DFLT_TIMEOUT?
   resolution should be explained at Javadoc

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureIgniteToKafka.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverImpl;
+import org.apache.ignite.internal.cdc.ChangeDataCapture;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+
+/**
+ * Change Data Consumer that streams all data changes to Kafka topic.
+ * {@link ChangeDataCaptureEvent} spread across Kafka topic partitions with 
{@code {ignite_partition} % {kafka_topic_count}} formula.
+ * Consumer will just fail in case of any error during write. Fail of consumer 
will lead to the fail of {@code ignite-cdc} application.
+ * It expected that {@code ignite-cdc} will be configured for automatic 
restarts with the OS tool to failover temporary errors
+ * such as Kafka unavailability or network issues.
+ *
+ * If you have plans to apply written messages to the other Ignite cluster in 
active-active manner,
+ * e.g. concurrent updates of the same entry in other cluster is possible,
+ * please, be aware of {@link CacheVersionConflictResolverImpl} conflict 
resolved.
+ * Configuration of {@link CacheVersionConflictResolverImpl} can be found in 
{@link ChangeDataCaptureKafkaToIgnite} documentation.
+ *
+ * @see ChangeDataCapture
+ * @see ChangeDataCaptureKafkaToIgnite
+ * @see CacheVersionConflictResolverImpl
+ */
+public class ChangeDataCaptureIgniteToKafka implements 
ChangeDataCaptureConsumer {
+    /** Log. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Kafka producer to stream events. */
+    private KafkaProducer<Integer, ChangeDataCaptureEvent> producer;
+
+    /** Handle only primary entry flag. */
+    private final boolean onlyPrimary;
+
+    /** Topic name. */
+    private final String topic;
+
+    /** Number Kafka topic partitions. */
+    private int kafkaPartitionsNum;
+
+    /** Cache IDs. */
+    private final Set<Integer> cachesIds;
+
+    /** Max batch size. */
+    private final int maxBatchSz;
+
+    /** Kafka properties. */
+    private Properties kafkaProps;
+
+    /** Count of sent messages.  */
+    private long cntSntMsgs;

Review comment:
       msgCnt?

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/Applier.java
##########
@@ -0,0 +1,347 @@
+/*
+ * 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.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.CacheEntryVersion;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl;
+import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
+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.ignite.internal.util.typedef.internal.U;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.WakeupException;
+
+/**
+ * Thread that polls message from the Kafka topic partitions and applies those 
messages to the Ignite caches.
+ * It expected that messages was written to the Kafka by the {@link 
ChangeDataCaptureIgniteToKafka} CDC consumer.
+ * <p>
+ * Each applier receive set of Kafka topic partitions to read and caches to 
process.
+ * Applier creates consumer per partition because Kafka consumer reads not 
fair,
+ * consumer reads messages from specific partition while there is new messages 
in specific partition.
+ * See <a href=
+ * 
"https://cwiki.apache.org/confluence/display/KAFKA/KIP-387%3A+Fair+Message+Consumption+Across+Partitions+in+KafkaConsumer";>KIP-387</a>
+ * and <a 
href="https://issues.apache.org/jira/browse/KAFKA-3932";>KAFKA-3932</a> for 
further information.
+ * All consumers should belongs to the same consumer-group to ensure 
consistent reading.
+ * Applier polls messages from each consumer in round-robin fashion.
+ * <p>
+ * Messages applied to Ignite using {@link 
IgniteInternalCache#putAllConflict(Map)}, {@link 
IgniteInternalCache#removeAllConflict(Map)}
+ * these methods allows to provide {@link GridCacheVersion} of the entry to 
the Ignite so in case update conflicts they can be resolved
+ * by the {@link CacheVersionConflictResolver}.
+ * <p>
+ * In case of any error during read applier just fail.
+ * Fail of any applier will lead to the fail of {@link 
ChangeDataCaptureKafkaToIgnite} application.
+ * It expected that application will be configured for automatic restarts with 
the OS tool to failover temporary errors
+ * such as Kafka or Ignite unavailability.
+ *
+ * @see ChangeDataCaptureKafkaToIgnite
+ * @see ChangeDataCaptureIgniteToKafka
+ * @see IgniteInternalCache#putAllConflict(Map)
+ * @see IgniteInternalCache#removeAllConflict(Map)
+ * @see CacheVersionConflictResolver
+ * @see GridCacheVersion
+ * @see ChangeDataCaptureEvent
+ * @see CacheEntryVersion
+ */
+class Applier implements Runnable, AutoCloseable {

Review comment:
       CdcApplier?
   Class name meaning can be simplified by prefixes.
   Let's call all the classes with prefixes.

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureKafkaToIgnite.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.ignite.cdc.conflictplugin.CacheConflictResolutionManagerImpl;
+import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverImpl;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.cdc.ChangeDataCapture;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.serialization.IntegerDeserializer;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.cdc.Utils.property;
+
+/**
+ * Main class of Kafka to Ignite application.
+ * This application is counterpart of {@link ChangeDataCaptureIgniteToKafka} 
Change Data Capture consumer.
+ * Application runs several {@link Applier} thread to read Kafka topic 
partitions and apply {@link ChangeDataCaptureEvent} to Ignite.
+ * <p>
+ * Each applier receive even number of kafka topic partition to read.
+ * <p>
+ * In case of any error during read applier just fail. Fail of any applier 
will lead to the fail of whole application.
+ * It expected that application will be configured for automatic restarts with 
the OS tool to failover temporary errors
+ * such as Kafka or Ignite unavailability.
+ * <p>
+ * To resolve possible update conflicts(in case of concurrent update in source 
and destination Ignite clusters)
+ * real-world deployments should use some conflict resolver, for example 
{@link CacheVersionConflictResolverImpl}.
+ * Example of Ignite configuration with the conflict resolver:
+ * <pre>
+ * {@code
+ * CacheVersionConflictResolverCachePluginProvider conflictPlugin = new 
CacheVersionConflictResolverCachePluginProvider();
+ *
+ * conflictPlugin.setClusterId(clusterId); // Cluster id.
+ * conflictPlugin.setCaches(new HashSet<>(Arrays.asList("my-cache", 
"some-other-cache"))); // Caches to replicate.
+ *
+ * IgniteConfiguration cfg = ...;
+ *
+ * cfg.setPluginProviders(conflictPlugin);
+ * }
+ * </pre>
+ * Please, see {@link CacheConflictResolutionManagerImpl} for additional 
information.
+ *
+ * Properties list:
+ * <ul>
+ *  <li>{@link #KAFKA_TO_IGNITE_THREAD_COUNT} - count of {@link Applier} 
threads.</li>
+ *  <li>{@link #IGNITE_TO_KAFKA_TOPIC} - Kafka topic name if not provided in 
constructor.</li>
+ * </ul>
+ *
+ * @see ChangeDataCapture
+ * @see ChangeDataCaptureIgniteToKafka
+ * @see ChangeDataCaptureEvent
+ * @see Applier
+ * @see CacheConflictResolutionManagerImpl
+ */
+public class ChangeDataCaptureKafkaToIgnite implements Runnable {
+    /** Property to define number of {@link Applier} threads. */
+    private static final String KAFKA_TO_IGNITE_THREAD_COUNT = 
"kafka.to.ignite.thread.count";
+
+    /** Ignite to Kafka topic name. */
+    public static final String IGNITE_TO_KAFKA_TOPIC = "ignite.to.kafka.topic";

Review comment:
       KAFKA_TOPIC
   
   why ignite TO kafka?

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureIgniteToKafka.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverImpl;
+import org.apache.ignite.internal.cdc.ChangeDataCapture;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+
+/**
+ * Change Data Consumer that streams all data changes to Kafka topic.
+ * {@link ChangeDataCaptureEvent} spread across Kafka topic partitions with 
{@code {ignite_partition} % {kafka_topic_count}} formula.
+ * Consumer will just fail in case of any error during write. Fail of consumer 
will lead to the fail of {@code ignite-cdc} application.
+ * It expected that {@code ignite-cdc} will be configured for automatic 
restarts with the OS tool to failover temporary errors
+ * such as Kafka unavailability or network issues.
+ *
+ * If you have plans to apply written messages to the other Ignite cluster in 
active-active manner,
+ * e.g. concurrent updates of the same entry in other cluster is possible,
+ * please, be aware of {@link CacheVersionConflictResolverImpl} conflict 
resolved.
+ * Configuration of {@link CacheVersionConflictResolverImpl} can be found in 
{@link ChangeDataCaptureKafkaToIgnite} documentation.
+ *
+ * @see ChangeDataCapture
+ * @see ChangeDataCaptureKafkaToIgnite
+ * @see CacheVersionConflictResolverImpl
+ */
+public class ChangeDataCaptureIgniteToKafka implements 
ChangeDataCaptureConsumer {
+    /** Log. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Kafka producer to stream events. */
+    private KafkaProducer<Integer, ChangeDataCaptureEvent> producer;
+
+    /** Handle only primary entry flag. */
+    private final boolean onlyPrimary;
+
+    /** Topic name. */
+    private final String topic;
+
+    /** Number Kafka topic partitions. */
+    private int kafkaPartitionsNum;
+
+    /** Cache IDs. */
+    private final Set<Integer> cachesIds;
+
+    /** Max batch size. */
+    private final int maxBatchSz;
+
+    /** Kafka properties. */
+    private Properties kafkaProps;
+
+    /** Count of sent messages.  */
+    private long cntSntMsgs;
+
+    /**
+     * @param topic Topic name.
+     * @param caches Cache names.
+     * @param maxBatchSz Maximum count of concurrently
+     * @param onlyPrimary If {@code true} then stream only events from 
primaries.
+     * @param kafkaProps Kafpa properties.
+     */
+    public ChangeDataCaptureIgniteToKafka(String topic, Set<String> caches, 
int maxBatchSz, boolean onlyPrimary, Properties kafkaProps) {
+        assert caches != null && !caches.isEmpty();
+
+        this.topic = topic;
+        this.onlyPrimary = onlyPrimary;
+        this.kafkaProps = kafkaProps;
+        this.maxBatchSz = maxBatchSz;
+
+        cachesIds = caches.stream()
+            .mapToInt(CU::cacheId)
+            .boxed()
+            .collect(Collectors.toSet());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onEvents(Iterator<ChangeDataCaptureEvent> evts) {
+        List<Future<RecordMetadata>> futs = new ArrayList<>();
+
+        while (evts.hasNext() && futs.size() <= maxBatchSz) {
+            ChangeDataCaptureEvent evt = evts.next();
+
+            if (onlyPrimary && !evt.primary())
+                continue;
+
+            if (evt.version().otherClusterVersion() != null)
+                continue;
+
+            if (!cachesIds.isEmpty() && !cachesIds.contains(evt.cacheId()))
+                continue;
+
+            cntSntMsgs++;
+
+            futs.add(producer.send(new ProducerRecord<>(
+                topic,
+                evt.partition() % kafkaPartitionsNum,
+                evt.cacheId(),
+                evt
+            )));
+        }
+
+        try {
+            for (Future<RecordMetadata> fut : futs)
+                fut.get(KafkaUtils.TIMEOUT_MIN, TimeUnit.MINUTES);
+        }
+        catch (InterruptedException | ExecutionException | TimeoutException e) 
{
+            throw new RuntimeException(e);
+        }
+
+        if (log.isDebugEnabled())
+            log.debug("Events processed [cntSntMsgs=" + cntSntMsgs + ']');
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        try {
+            kafkaPartitionsNum = KafkaUtils.initTopic(topic, kafkaProps);
+
+            producer = new KafkaProducer<>(kafkaProps);
+
+            log.info("Ignite To Kafka started [topic=" + topic + ", 
onlyPrimary=" + onlyPrimary + ", cacheIds=" + cachesIds + ']');

Review comment:
       Ignite To Kafka consumer/producer?

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureIgniteToKafka.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverImpl;
+import org.apache.ignite.internal.cdc.ChangeDataCapture;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+
+/**
+ * Change Data Consumer that streams all data changes to Kafka topic.
+ * {@link ChangeDataCaptureEvent} spread across Kafka topic partitions with 
{@code {ignite_partition} % {kafka_topic_count}} formula.
+ * Consumer will just fail in case of any error during write. Fail of consumer 
will lead to the fail of {@code ignite-cdc} application.
+ * It expected that {@code ignite-cdc} will be configured for automatic 
restarts with the OS tool to failover temporary errors
+ * such as Kafka unavailability or network issues.
+ *
+ * If you have plans to apply written messages to the other Ignite cluster in 
active-active manner,
+ * e.g. concurrent updates of the same entry in other cluster is possible,
+ * please, be aware of {@link CacheVersionConflictResolverImpl} conflict 
resolved.
+ * Configuration of {@link CacheVersionConflictResolverImpl} can be found in 
{@link ChangeDataCaptureKafkaToIgnite} documentation.
+ *
+ * @see ChangeDataCapture
+ * @see ChangeDataCaptureKafkaToIgnite
+ * @see CacheVersionConflictResolverImpl
+ */
+public class ChangeDataCaptureIgniteToKafka implements 
ChangeDataCaptureConsumer {

Review comment:
       KafkaChangeDataCaptureConsumer?

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/KafkaUtils.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
+
+import static org.apache.ignite.cdc.Utils.property;
+
+/** Kafka Utils. */
+public class KafkaUtils {
+    /** Default kafka request timeout. */
+    public static final int TIMEOUT_MIN = 1;
+
+    /** Ignite to Kafka topic partitions number. */
+    public static final String IGNITE_TO_KAFKA_NUM_PARTITIONS = 
"ignite.to.kafka.numpartitions";
+
+    /** Ignite to Kafka topic partitions number. */
+    public static final String IGNITE_TO_KAFKA_REPLICATION_FACTOR = 
"ignite.to.kafka.replication.factor";
+
+    /**
+     * Initialize Kafka topic.
+     *
+     * @param props Properties.
+     */
+    public static int initTopic(String topic, Properties props)
+        throws InterruptedException, ExecutionException, TimeoutException {
+        try (AdminClient adminCli = AdminClient.create(props)) {
+            return createTopic(
+                topic,
+                Integer.parseInt(property(IGNITE_TO_KAFKA_NUM_PARTITIONS, 
props, "32")),
+                property(IGNITE_TO_KAFKA_REPLICATION_FACTOR, props, "1"),

Review comment:
       let's make defaults constants

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureKafkaToIgnite.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.ignite.cdc.conflictplugin.CacheConflictResolutionManagerImpl;
+import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverImpl;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.cdc.ChangeDataCapture;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.serialization.IntegerDeserializer;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.cdc.Utils.property;
+
+/**
+ * Main class of Kafka to Ignite application.
+ * This application is counterpart of {@link ChangeDataCaptureIgniteToKafka} 
Change Data Capture consumer.
+ * Application runs several {@link Applier} thread to read Kafka topic 
partitions and apply {@link ChangeDataCaptureEvent} to Ignite.
+ * <p>
+ * Each applier receive even number of kafka topic partition to read.
+ * <p>
+ * In case of any error during read applier just fail. Fail of any applier 
will lead to the fail of whole application.
+ * It expected that application will be configured for automatic restarts with 
the OS tool to failover temporary errors
+ * such as Kafka or Ignite unavailability.
+ * <p>
+ * To resolve possible update conflicts(in case of concurrent update in source 
and destination Ignite clusters)
+ * real-world deployments should use some conflict resolver, for example 
{@link CacheVersionConflictResolverImpl}.
+ * Example of Ignite configuration with the conflict resolver:
+ * <pre>
+ * {@code
+ * CacheVersionConflictResolverCachePluginProvider conflictPlugin = new 
CacheVersionConflictResolverCachePluginProvider();
+ *
+ * conflictPlugin.setClusterId(clusterId); // Cluster id.
+ * conflictPlugin.setCaches(new HashSet<>(Arrays.asList("my-cache", 
"some-other-cache"))); // Caches to replicate.
+ *
+ * IgniteConfiguration cfg = ...;
+ *
+ * cfg.setPluginProviders(conflictPlugin);
+ * }
+ * </pre>
+ * Please, see {@link CacheConflictResolutionManagerImpl} for additional 
information.
+ *
+ * Properties list:
+ * <ul>
+ *  <li>{@link #KAFKA_TO_IGNITE_THREAD_COUNT} - count of {@link Applier} 
threads.</li>
+ *  <li>{@link #IGNITE_TO_KAFKA_TOPIC} - Kafka topic name if not provided in 
constructor.</li>
+ * </ul>
+ *
+ * @see ChangeDataCapture
+ * @see ChangeDataCaptureIgniteToKafka
+ * @see ChangeDataCaptureEvent
+ * @see Applier
+ * @see CacheConflictResolutionManagerImpl
+ */
+public class ChangeDataCaptureKafkaToIgnite implements Runnable {
+    /** Property to define number of {@link Applier} threads. */
+    private static final String KAFKA_TO_IGNITE_THREAD_COUNT = 
"kafka.to.ignite.thread.count";

Review comment:
       THREAD_COUNT

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureIgniteToKafka.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverImpl;
+import org.apache.ignite.internal.cdc.ChangeDataCapture;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+
+/**
+ * Change Data Consumer that streams all data changes to Kafka topic.
+ * {@link ChangeDataCaptureEvent} spread across Kafka topic partitions with 
{@code {ignite_partition} % {kafka_topic_count}} formula.
+ * Consumer will just fail in case of any error during write. Fail of consumer 
will lead to the fail of {@code ignite-cdc} application.
+ * It expected that {@code ignite-cdc} will be configured for automatic 
restarts with the OS tool to failover temporary errors
+ * such as Kafka unavailability or network issues.
+ *
+ * If you have plans to apply written messages to the other Ignite cluster in 
active-active manner,
+ * e.g. concurrent updates of the same entry in other cluster is possible,
+ * please, be aware of {@link CacheVersionConflictResolverImpl} conflict 
resolved.
+ * Configuration of {@link CacheVersionConflictResolverImpl} can be found in 
{@link ChangeDataCaptureKafkaToIgnite} documentation.
+ *
+ * @see ChangeDataCapture
+ * @see ChangeDataCaptureKafkaToIgnite
+ * @see CacheVersionConflictResolverImpl
+ */
+public class ChangeDataCaptureIgniteToKafka implements 
ChangeDataCaptureConsumer {
+    /** Log. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Kafka producer to stream events. */
+    private KafkaProducer<Integer, ChangeDataCaptureEvent> producer;
+
+    /** Handle only primary entry flag. */
+    private final boolean onlyPrimary;
+
+    /** Topic name. */
+    private final String topic;
+
+    /** Number Kafka topic partitions. */
+    private int kafkaPartitionsNum;
+
+    /** Cache IDs. */
+    private final Set<Integer> cachesIds;
+
+    /** Max batch size. */
+    private final int maxBatchSz;
+
+    /** Kafka properties. */
+    private Properties kafkaProps;
+
+    /** Count of sent messages.  */
+    private long cntSntMsgs;
+
+    /**
+     * @param topic Topic name.
+     * @param caches Cache names.
+     * @param maxBatchSz Maximum count of concurrently
+     * @param onlyPrimary If {@code true} then stream only events from 
primaries.
+     * @param kafkaProps Kafpa properties.
+     */
+    public ChangeDataCaptureIgniteToKafka(String topic, Set<String> caches, 
int maxBatchSz, boolean onlyPrimary, Properties kafkaProps) {
+        assert caches != null && !caches.isEmpty();
+
+        this.topic = topic;
+        this.onlyPrimary = onlyPrimary;
+        this.kafkaProps = kafkaProps;
+        this.maxBatchSz = maxBatchSz;
+
+        cachesIds = caches.stream()
+            .mapToInt(CU::cacheId)
+            .boxed()
+            .collect(Collectors.toSet());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onEvents(Iterator<ChangeDataCaptureEvent> evts) {
+        List<Future<RecordMetadata>> futs = new ArrayList<>();
+
+        while (evts.hasNext() && futs.size() <= maxBatchSz) {
+            ChangeDataCaptureEvent evt = evts.next();
+
+            if (onlyPrimary && !evt.primary())
+                continue;
+
+            if (evt.version().otherClusterVersion() != null)
+                continue;
+
+            if (!cachesIds.isEmpty() && !cachesIds.contains(evt.cacheId()))
+                continue;
+
+            cntSntMsgs++;
+
+            futs.add(producer.send(new ProducerRecord<>(
+                topic,
+                evt.partition() % kafkaPartitionsNum,
+                evt.cacheId(),
+                evt
+            )));
+        }
+
+        try {
+            for (Future<RecordMetadata> fut : futs)
+                fut.get(KafkaUtils.TIMEOUT_MIN, TimeUnit.MINUTES);
+        }
+        catch (InterruptedException | ExecutionException | TimeoutException e) 
{
+            throw new RuntimeException(e);
+        }
+
+        if (log.isDebugEnabled())
+            log.debug("Events processed [cntSntMsgs=" + cntSntMsgs + ']');
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        try {
+            kafkaPartitionsNum = KafkaUtils.initTopic(topic, kafkaProps);
+
+            producer = new KafkaProducer<>(kafkaProps);
+
+            log.info("Ignite To Kafka started [topic=" + topic + ", 
onlyPrimary=" + onlyPrimary + ", cacheIds=" + cachesIds + ']');
+        }
+        catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Sets kafka properties.
+     *
+     * @param kafkaProps Kafka properties.
+     */
+    public void setKafkaProps(Properties kafkaProps) {
+        this.kafkaProps = kafkaProps;
+    }

Review comment:
       Could we have implementation methods located close to each other.
   Let's relocate this method to the end of the file. 

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureIgniteToKafka.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverImpl;
+import org.apache.ignite.internal.cdc.ChangeDataCapture;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+
+/**
+ * Change Data Consumer that streams all data changes to Kafka topic.
+ * {@link ChangeDataCaptureEvent} spread across Kafka topic partitions with 
{@code {ignite_partition} % {kafka_topic_count}} formula.
+ * Consumer will just fail in case of any error during write. Fail of consumer 
will lead to the fail of {@code ignite-cdc} application.
+ * It expected that {@code ignite-cdc} will be configured for automatic 
restarts with the OS tool to failover temporary errors
+ * such as Kafka unavailability or network issues.
+ *
+ * If you have plans to apply written messages to the other Ignite cluster in 
active-active manner,
+ * e.g. concurrent updates of the same entry in other cluster is possible,
+ * please, be aware of {@link CacheVersionConflictResolverImpl} conflict 
resolved.
+ * Configuration of {@link CacheVersionConflictResolverImpl} can be found in 
{@link ChangeDataCaptureKafkaToIgnite} documentation.
+ *
+ * @see ChangeDataCapture
+ * @see ChangeDataCaptureKafkaToIgnite
+ * @see CacheVersionConflictResolverImpl
+ */
+public class ChangeDataCaptureIgniteToKafka implements 
ChangeDataCaptureConsumer {
+    /** Log. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Kafka producer to stream events. */
+    private KafkaProducer<Integer, ChangeDataCaptureEvent> producer;
+
+    /** Handle only primary entry flag. */
+    private final boolean onlyPrimary;
+
+    /** Topic name. */
+    private final String topic;
+
+    /** Number Kafka topic partitions. */
+    private int kafkaPartitionsNum;
+
+    /** Cache IDs. */
+    private final Set<Integer> cachesIds;
+
+    /** Max batch size. */
+    private final int maxBatchSz;

Review comment:
       maxBatchSize?

##########
File path: 
modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureIgniteToKafka.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverImpl;
+import org.apache.ignite.internal.cdc.ChangeDataCapture;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+
+/**
+ * Change Data Consumer that streams all data changes to Kafka topic.
+ * {@link ChangeDataCaptureEvent} spread across Kafka topic partitions with 
{@code {ignite_partition} % {kafka_topic_count}} formula.
+ * Consumer will just fail in case of any error during write. Fail of consumer 
will lead to the fail of {@code ignite-cdc} application.
+ * It expected that {@code ignite-cdc} will be configured for automatic 
restarts with the OS tool to failover temporary errors
+ * such as Kafka unavailability or network issues.
+ *
+ * If you have plans to apply written messages to the other Ignite cluster in 
active-active manner,
+ * e.g. concurrent updates of the same entry in other cluster is possible,
+ * please, be aware of {@link CacheVersionConflictResolverImpl} conflict 
resolved.
+ * Configuration of {@link CacheVersionConflictResolverImpl} can be found in 
{@link ChangeDataCaptureKafkaToIgnite} documentation.
+ *
+ * @see ChangeDataCapture
+ * @see ChangeDataCaptureKafkaToIgnite
+ * @see CacheVersionConflictResolverImpl
+ */
+public class ChangeDataCaptureIgniteToKafka implements 
ChangeDataCaptureConsumer {
+    /** Log. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Kafka producer to stream events. */
+    private KafkaProducer<Integer, ChangeDataCaptureEvent> producer;
+
+    /** Handle only primary entry flag. */
+    private final boolean onlyPrimary;
+
+    /** Topic name. */
+    private final String topic;
+
+    /** Number Kafka topic partitions. */
+    private int kafkaPartitionsNum;

Review comment:
       amount




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to