nizhikov commented on a change in pull request #49: URL: https://github.com/apache/ignite-extensions/pull/49#discussion_r655531203
########## File path: modules/cdc-ext/src/main/java/org/apache/ignite/cdc/kafka/KafkaToIgniteCdcStreamerApplier.java ########## @@ -0,0 +1,253 @@ +/* + * 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.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.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.CacheEntryVersion; +import org.apache.ignite.cdc.ChangeDataCaptureEvent; +import org.apache.ignite.cdc.CdcEventsApplier; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +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.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 extends CdcEventsApplier 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; + + /** Kafka properties. */ + private final Properties kafkaProps; + + /** Topic to read. */ + private final String topic; + + /** Lower kafka partition (inclusive). */ + private final int kafkaPartFrom; + + /** Higher kafka partition (exclusive). */ + private final int kafkaPartTo; + + /** Caches ids to read. */ + private final Set<Integer> caches; + + /** Consumers. */ + private final List<KafkaConsumer<Integer, byte[]>> cnsmrs = new ArrayList<>(); + + /** */ + private final AtomicLong rcvdEvts = new AtomicLong(); + + /** + * @param ign Ignite instance. + * @param log Logger. + * @param kafkaProps Kafka properties. + * @param topic Topic name. + * @param kafkaPartFrom Read from partition. + * @param kafkaPartTo Read to partition. + * @param caches Cache ids. + * @param maxBatchSize Maximum batch size. + * @param closed Closed flag. + */ + public KafkaToIgniteCdcStreamerApplier( + IgniteEx ign, + IgniteLogger log, + Properties kafkaProps, + String topic, + int kafkaPartFrom, + int kafkaPartTo, + Set<Integer> caches, + int maxBatchSize, + AtomicBoolean closed + ) { + super(maxBatchSize); + + this.ign = ign; + this.kafkaProps = kafkaProps; + this.topic = topic; + this.kafkaPartFrom = kafkaPartFrom; + this.kafkaPartTo = kafkaPartTo; + this.caches = caches; + this.closed = closed; + this.log = log.getLogger(KafkaToIgniteCdcStreamerApplier.class); + } + + /** {@inheritDoc} */ + @Override public void run() { + U.setCurrentIgniteName(ign.name()); + + try { + for (int kafkaPart = kafkaPartFrom; kafkaPart < kafkaPartTo; kafkaPart++) { + KafkaConsumer<Integer, byte[]> cnsmr = new KafkaConsumer<>(kafkaProps); + + cnsmr.assign(Collections.singleton(new TopicPartition(topic, kafkaPart))); + + cnsmrs.add(cnsmr); + } + + Iterator<KafkaConsumer<Integer, byte[]>> cnsmrIter = Collections.emptyIterator(); + + while (!closed.get()) { + if (!cnsmrIter.hasNext()) + cnsmrIter = cnsmrs.iterator(); + + poll(cnsmrIter.next()); + } + } + catch (WakeupException e) { + if (!closed.get()) + log.error("Applier wakeup error!", e); + } + catch (Throwable e) { + log.error("Applier error!", e); + + closed.set(true); Review comment: I don't think so. Why? All other appliers will try to perform last poll and will finish after. -- 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]
