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



##########
File path: 
modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java
##########
@@ -0,0 +1,357 @@
+/*
+ * 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.kafka;
+
+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.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.cdc.ChangeDataCaptureEvent;
+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 
IgniteToKafkaCdcStreamer} Change Data Capture consumer.
+ * <p>
+ * Each {@code 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 
KafkaToIgniteCdcStreamer} 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 KafkaToIgniteCdcStreamer
+ * @see IgniteToKafkaCdcStreamer
+ * @see IgniteInternalCache#putAllConflict(Map)
+ * @see IgniteInternalCache#removeAllConflict(Map)
+ * @see CacheVersionConflictResolver
+ * @see GridCacheVersion
+ * @see ChangeDataCaptureEvent
+ * @see CacheEntryVersion
+ */
+class KafkaToIgniteCdcStreamerApplier implements Runnable, AutoCloseable {
+    /** */
+    public static final int DFLT_REQ_TIMEOUT = 3;
+
+    /** Ignite instance. */
+    private final IgniteEx ign;
+
+    /** Log. */
+    private final IgniteLogger log;
+
+    /** Closed flag. Shared between all appliers. */
+    private final AtomicBoolean closed;
+
+    /** Caches. */
+    private final Map<Integer, IgniteInternalCache<BinaryObject, 
BinaryObject>> ignCaches = new HashMap<>();
+
+    /** Kafka properties. */
+    private final Properties kafkaProps;
+
+    /** Maximum batch size. */
+    private final int maxBatchSize;
+
+    /** Topic to read. */
+    private final String topic;
+
+    /** Lower kafka partition (inclusive). */
+    private final int kafkaPartFrom;
+
+    /** Higher kafka partition (exclusive). */
+    private final int kafkaPartTo;

Review comment:
       It's not obvious from naming that one is inclusive but another is 
exclusive.
   Lets, at least make both similar.

##########
File path: 
modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamer.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.kafka;
+
+import java.util.ArrayList;
+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.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cdc.ChangeDataCaptureEvent;
+import 
org.apache.ignite.cdc.conflictresolve.CacheConflictResolutionManagerImpl;
+import org.apache.ignite.cdc.conflictresolve.CacheVersionConflictResolverImpl;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.cdc.ChangeDataCapture;
+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.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.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
+
+/**
+ * Main class of Kafka to Ignite application.
+ * This application is counterpart of {@link IgniteToKafkaCdcStreamer} Change 
Data Capture consumer.
+ * Application runs several {@link KafkaToIgniteCdcStreamerApplier} 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.
+ *
+ * @see ChangeDataCapture
+ * @see IgniteToKafkaCdcStreamer
+ * @see ChangeDataCaptureEvent
+ * @see KafkaToIgniteCdcStreamerApplier
+ * @see CacheConflictResolutionManagerImpl
+ */
+public class KafkaToIgniteCdcStreamer implements Runnable {
+    /** Ignite configuration. */
+    private final IgniteConfiguration igniteCfg;
+
+    /** Kafka consumer properties. */
+    private final Properties kafkaProps;
+
+    /** Streamer configuration. */
+    private final KafkaToIgniteCdcStreamerConfiguration streamerCfg;
+
+    /** Executor service to run {@link KafkaToIgniteCdcStreamerApplier} 
instances. */
+    private final ExecutorService execSvc;
+
+    /** Appliers. */
+    private final List<KafkaToIgniteCdcStreamerApplier> appliers;
+
+    /**
+     * @param igniteCfg Ignite configuration.
+     * @param kafkaProps Kafka properties.
+     * @param streamerCfg Streamer configuration.
+     */
+    public KafkaToIgniteCdcStreamer(
+        IgniteConfiguration igniteCfg,
+        Properties kafkaProps,
+        KafkaToIgniteCdcStreamerConfiguration streamerCfg
+    ) {
+        this.igniteCfg = igniteCfg;
+        this.kafkaProps = kafkaProps;
+        this.streamerCfg = streamerCfg;
+
+        appliers = new ArrayList<>(streamerCfg.getThreadCount());
+
+        execSvc = Executors.newFixedThreadPool(streamerCfg.getThreadCount(), 
new ThreadFactory() {
+            private final AtomicInteger cntr = new AtomicInteger();
+
+            @Override public Thread newThread(@NotNull Runnable r) {
+                Thread th = new Thread(r);
+
+                th.setName("applier-thread-" + cntr.getAndIncrement());
+
+                return th;
+            }
+        });

Review comment:
       ExecutorSerice for a single run looks overcomplicated to me.
   Why not just Thread.start/join?




-- 
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