nizhikov commented on a change in pull request #49: URL: https://github.com/apache/ignite-extensions/pull/49#discussion_r648049096
########## 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: I like the current naming: `IgniteToKafka` streams changes from Ignite to Kafka. `KafkaToIgnite` streams changes from Kafka to Ignite. What's wrong with it? Please, not, this `Runnable` is a main class of the application. ########## 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: I like the current naming: `IgniteToKafka` streams changes from Ignite to Kafka. `KafkaToIgnite` streams changes from Kafka to Ignite. What's wrong with it? Please, note, this `Runnable` is a main class of the application. -- 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]
