anton-vinogradov commented on a change in pull request #49: URL: https://github.com/apache/ignite-extensions/pull/49#discussion_r649006352
########## File path: modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/conflictplugin/CacheConflictResolutionManagerImpl.java ########## @@ -0,0 +1,89 @@ +/* + * 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.conflictplugin; Review comment: org.apache.ignite.cdc.conflict(s,resolve) ########## File path: modules/cdc-to-kafka-ext/src/test/java/org/apache/ignite/cdc/KafkaToIgniteLoaderTest.java ########## @@ -0,0 +1,49 @@ +/* + * 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 org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; + +import static org.apache.ignite.cdc.KafkaToIgniteLoader.loadKafkaToIgniteStreamer; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; +import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause; + +/** Tests load {@link KafkaToIgniteCdcStreamer} from Srping xml file. */ +public class KafkaToIgniteLoaderTest extends GridCommonAbstractTest { + /** */ + @Test + public void testLoadConfig() throws Exception { + assertThrows( + null, + () -> loadKafkaToIgniteStreamer("kafka-to-ignite-2.xml"), + IgniteCheckedException.class, + "Exact 1 IgniteConfiguration should be defined. Found 2" + ); + + assertThrowsWithCause( + () -> loadKafkaToIgniteStreamer("kafka-to-ignite-3.xml"), Review comment: the problem should be explained at cfg name, not just 3 ########## File path: modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/IgniteToKafkaCdcStreamer.java ########## @@ -0,0 +1,173 @@ +/* + * 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 KafkaToIgniteCdcStreamer} documentation. + * + * @see ChangeDataCapture + * @see KafkaToIgniteCdcStreamer + * @see CacheVersionConflictResolverImpl + */ +public class IgniteToKafkaCdcStreamer implements ChangeDataCaptureConsumer { + /** Default kafka request timeout. */ + public static final int DFLT_REQ_TIMEOUT_MIN = 1; + + /** 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; + + /** Kafka topic partitions count. */ + private final int kafkaParts; + + /** Kafka properties. */ + private final Properties kafkaProps; + + /** Cache IDs. */ + private final Set<Integer> cachesIds; + + /** Max batch size. */ + private final int maxBatchSize; + + /** Count of sent messages. */ + private long msgCnt; + + /** + * @param topic Topic name. + * @param kafkaParts Kafka partitions count. + * @param caches Cache names. + * @param maxBatchSize Maximum count of concurrently. Review comment: > Maximum count of concurrently. Whan this means? ########## File path: modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/conflictplugin/CacheVersionConflictResolverImpl.java ########## @@ -0,0 +1,158 @@ +/* + * 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.conflictplugin; + +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.internal.processors.cache.CacheObjectValueContext; +import org.apache.ignite.internal.processors.cache.version.CacheVersionConflictResolver; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * This class implements simple conflict resolution algorithm. + * Algorithm decides which version of the entry should be used "new" or "old". + * The following steps performed: + * <ul> + * <li>If entry is freshly created then new version used - {@link GridCacheVersionedEntryEx#isStartVersion()}.</li> + * <li>If change made in this cluster then new version used - {@link GridCacheVersionedEntryEx#dataCenterId()}.</li> + * <li>If cluster of new entry equal to cluster of old entry + * then entry with the greater {@link GridCacheVersionedEntryEx#order()} used.</li> + * <li>If {@link #conflictResolveField} provided and field of new entry greater then new version used.</li> + * <li>If {@link #conflictResolveField} provided and field of old entry greater then old version used.</li> + * <li>Entry with the lower value of {@link GridCacheVersionedEntryEx#dataCenterId()} used.</li> + * </ul> + * + * Note, data center with lower value has greater priority e.g first (1) data center is main in case conflict can't be resolved + * automatically. + */ +public class CacheVersionConflictResolverImpl implements CacheVersionConflictResolver { + /** + * Cluster id. + * Note, cluster with lower value has greater priority e.g first (1) cluster is main in case conflict can't be resolved automatically. + */ + private final byte clusterId; + + /** + * Field for conflict resolve. + * Value of this field will be used to compare two entries in case of conflicting changes. + * Note, values of this field must implement {@link Comparable} interface. + * + * @see CacheVersionConflictResolverImpl + */ + private final String conflictResolveField; + + /** Logger. */ + private final IgniteLogger log; + + /** If {@code true} then conflict resolving with the value field enabled. */ + private boolean conflictResolveFieldEnabled; + + /** + * @param clusterId Data center id. + * @param conflictResolveField Field to resolve conflicts. + * @param log Logger. + */ + public CacheVersionConflictResolverImpl(byte clusterId, String conflictResolveField, IgniteLogger log) { + this.clusterId = clusterId; + this.conflictResolveField = conflictResolveField; + this.log = log; + + conflictResolveFieldEnabled = conflictResolveField != null; + } + + /** {@inheritDoc} */ + @Override public <K, V> GridCacheVersionConflictContext<K, V> resolve( + CacheObjectValueContext ctx, + GridCacheVersionedEntryEx<K, V> oldEntry, + GridCacheVersionedEntryEx<K, V> newEntry, + boolean atomicVerComparator + ) { + GridCacheVersionConflictContext<K, V> res = new GridCacheVersionConflictContext<>(ctx, oldEntry, newEntry); + + if (isUseNew(ctx, oldEntry, newEntry)) + res.useNew(); + else { + log.warning("Skip update due to the conflict[key=" + newEntry.key() + ",fromCluster=" + newEntry.dataCenterId() + + ",toCluster=" + oldEntry.dataCenterId() + ']'); + + res.useOld(); + } + + return res; + } + + /** + * @param ctx Context. + * @param oldEntry Old entry. + * @param newEntry New entry. + * @param <K> Key type. + * @param <V> Key type. + * @return {@code True} is should use new entry. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + private <K, V> boolean isUseNew( + CacheObjectValueContext ctx, + GridCacheVersionedEntryEx<K, V> oldEntry, + GridCacheVersionedEntryEx<K, V> newEntry + ) { + if (oldEntry.isStartVersion()) // New entry. + return true; + + if (newEntry.dataCenterId() == clusterId) // Update made on the same DC. + return true; + + if (oldEntry.dataCenterId() == newEntry.dataCenterId()) + return newEntry.order() > oldEntry.order(); // New version from the same DC. Review comment: version should be checked instead. ########## File path: modules/cdc-to-kafka-ext/src/test/java/org/apache/ignite/cdc/KafkaToIgniteLoaderTest.java ########## @@ -0,0 +1,49 @@ +/* + * 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 org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; + +import static org.apache.ignite.cdc.KafkaToIgniteLoader.loadKafkaToIgniteStreamer; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; +import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause; + +/** Tests load {@link KafkaToIgniteCdcStreamer} from Srping xml file. */ +public class KafkaToIgniteLoaderTest extends GridCommonAbstractTest { + /** */ + @Test + public void testLoadConfig() throws Exception { + assertThrows( + null, + () -> loadKafkaToIgniteStreamer("kafka-to-ignite-2.xml"), + IgniteCheckedException.class, + "Exact 1 IgniteConfiguration should be defined. Found 2" + ); + + assertThrowsWithCause( + () -> loadKafkaToIgniteStreamer("kafka-to-ignite-3.xml"), + IgniteCheckedException.class + ); Review comment: Exception message specification missed ########## File path: modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/IgniteToKafkaCdcStreamer.java ########## @@ -0,0 +1,173 @@ +/* + * 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 KafkaToIgniteCdcStreamer} documentation. + * + * @see ChangeDataCapture + * @see KafkaToIgniteCdcStreamer + * @see CacheVersionConflictResolverImpl + */ +public class IgniteToKafkaCdcStreamer implements ChangeDataCaptureConsumer { + /** Default kafka request timeout. */ + public static final int DFLT_REQ_TIMEOUT_MIN = 1; Review comment: DFLT_REQ_TIMEOUT as I told you before, the resolution should be specified at javadoc. ########## File path: modules/cdc-to-kafka-ext/src/test/java/org/apache/ignite/cdc/KafkaToIgniteLoaderTest.java ########## @@ -0,0 +1,49 @@ +/* + * 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 org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; + +import static org.apache.ignite.cdc.KafkaToIgniteLoader.loadKafkaToIgniteStreamer; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; +import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause; + +/** Tests load {@link KafkaToIgniteCdcStreamer} from Srping xml file. */ +public class KafkaToIgniteLoaderTest extends GridCommonAbstractTest { + /** */ + @Test + public void testLoadConfig() throws Exception { + assertThrows( + null, + () -> loadKafkaToIgniteStreamer("kafka-to-ignite-2.xml"), Review comment: the problem should be explained at cfg name, not just 2 ########## File path: modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/IgniteToKafkaCdcStreamer.java ########## @@ -0,0 +1,173 @@ +/* + * 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 KafkaToIgniteCdcStreamer} documentation. + * + * @see ChangeDataCapture + * @see KafkaToIgniteCdcStreamer + * @see CacheVersionConflictResolverImpl + */ +public class IgniteToKafkaCdcStreamer implements ChangeDataCaptureConsumer { + /** Default kafka request timeout. */ + public static final int DFLT_REQ_TIMEOUT_MIN = 1; + + /** 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; + + /** Kafka topic partitions count. */ + private final int kafkaParts; + + /** Kafka properties. */ + private final Properties kafkaProps; + + /** Cache IDs. */ + private final Set<Integer> cachesIds; + + /** Max batch size. */ + private final int maxBatchSize; + + /** Count of sent messages. */ + private long msgCnt; + + /** + * @param topic Topic name. + * @param kafkaParts Kafka partitions count. + * @param caches Cache names. + * @param maxBatchSize Maximum count of concurrently. + * @param onlyPrimary If {@code true} then stream only events from primaries. + * @param kafkaProps Kafka properties. + */ + public IgniteToKafkaCdcStreamer( + String topic, + int kafkaParts, + Set<String> caches, + int maxBatchSize, + boolean onlyPrimary, + Properties kafkaProps + ) { + assert caches != null && !caches.isEmpty(); + + this.topic = topic; + this.kafkaParts = kafkaParts; + this.onlyPrimary = onlyPrimary; + this.kafkaProps = kafkaProps; + this.maxBatchSize = maxBatchSize; + + 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() <= maxBatchSize) { Review comment: should be `<`? ########## File path: modules/cdc-to-kafka-ext/src/test/java/org/apache/ignite/cdc/CdcKafkaReplicationTest.java ########## @@ -0,0 +1,519 @@ +/* + * 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.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Objects; +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 java.util.concurrent.atomic.AtomicLong; +import java.util.stream.IntStream; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverPluginProvider; +import org.apache.ignite.cdc.serde.JavaObjectSerializer; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.utility.DockerImageName; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.KEYS_CNT; Review comment: please avoid such usages when the test is not an inheritor ########## File path: modules/cdc-to-kafka-ext/src/main/java/org/apache/ignite/cdc/serde/JavaObjectSerializer.java ########## @@ -0,0 +1,41 @@ +/* + * 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.serde; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import org.apache.kafka.common.serialization.Serializer; + +/** Java object serializer. */ +public class JavaObjectSerializer implements Serializer<Serializable> { Review comment: use IgniteUtils#toBytes instead ########## File path: modules/cdc-to-kafka-ext/src/test/java/org/apache/ignite/cdc/CdcKafkaReplicationTest.java ########## @@ -0,0 +1,523 @@ +/* + * 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.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Objects; +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 java.util.concurrent.atomic.AtomicLong; +import java.util.stream.IntStream; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cdc.conflictplugin.CacheVersionConflictResolverPluginProvider; +import org.apache.ignite.cdc.serde.JavaObjectSerializer; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; +import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.utility.DockerImageName; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.KEYS_CNT; +import static org.apache.ignite.cdc.IgniteToKafkaCdcStreamer.DFLT_REQ_TIMEOUT_MIN; +import static org.apache.ignite.cdc.KafkaToIgniteCdcStreamerConfiguration.DFLT_PARTS; +import static org.apache.ignite.cdc.KafkaToIgniteCdcStreamerConfiguration.DFLT_TOPIC; +import static org.apache.ignite.cluster.ClusterState.ACTIVE; +import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.DFLT_PORT_RANGE; +import static org.apache.ignite.testframework.GridTestUtils.getFieldValue; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; + +/** + * Tests for kafka replication. + */ +@RunWith(Parameterized.class) +public class CdcKafkaReplicationTest extends GridCommonAbstractTest { + /** Cache mode. */ + @Parameterized.Parameter + public CacheAtomicityMode cacheMode; + + /** */ + @Parameterized.Parameter(1) + public int backupCnt; + + /** @return Test parameters. */ + @Parameterized.Parameters(name = "cacheMode={0},backupCnt={1}") + public static Collection<?> parameters() { + return Arrays.asList(new Object[][] { + {ATOMIC, 0}, + {ATOMIC, 1}, + {TRANSACTIONAL, 0}, + {TRANSACTIONAL, 1} + }); + } + + /** */ + public static final String SRC_DEST_TOPIC = "source-dest"; + + /** */ + public static final String DEST_SRC_TOPIC = "dest-source"; + + /** */ + public static final String AP_CACHE = "active-passive-cache"; + + /** */ + public static final String ACTIVE_ACTIVE_CACHE = "active-active-cache"; + + /** */ + public static final byte SRC_CLUSTER_ID = 26; + + /** */ + public static final byte DEST_CLUSTER_ID = 27; + + /** */ + public static final int BOTH_EXISTS = 1; + + /** */ + public static final int BOTH_REMOVED = 2; + + /** */ + public static final int ANY_SAME_STATE = 3; + + /** */ + public static final AtomicLong REQUEST_ID = new AtomicLong(); + + /** */ + private static Properties props; + + /** */ + private static IgniteEx[] srcCluster; + + /** */ + private static IgniteConfiguration srcClusterCliCfg; + + /** */ + private static IgniteEx[] destCluster; + + /** */ + private static IgniteConfiguration destClusterCliCfg; + + /** */ + @ClassRule + public static KafkaContainer kafka = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:5.4.3")); + + /** */ + private int commPort = TcpCommunicationSpi.DFLT_PORT; + + /** */ + private int discoPort = TcpDiscoverySpi.DFLT_PORT; + + /** */ + private byte clusterId = SRC_CLUSTER_ID; + + /** */ + private static final ThreadLocal<FastCrc> crc = new ThreadLocal<>().withInitial(FastCrc::new); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + CacheVersionConflictResolverPluginProvider<?> cfgPlugin = new CacheVersionConflictResolverPluginProvider<>(); + + cfgPlugin.setClusterId(clusterId); + cfgPlugin.setCaches(new HashSet<>(Collections.singletonList(ACTIVE_ACTIVE_CACHE))); + cfgPlugin.setConflictResolveField("reqId"); + + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName) + .setDiscoverySpi(new TcpDiscoverySpi() + .setLocalPort(discoPort) + .setLocalPortRange(DFLT_PORT_RANGE) + .setIpFinder(new TcpDiscoveryVmIpFinder() {{ + setAddresses(Collections.singleton("127.0.0.1:" + discoPort + ".." + (discoPort + DFLT_PORT_RANGE))); + }})) + .setCommunicationSpi(new TcpCommunicationSpi() + .setLocalPort(commPort) + .setLocalPortRange(DFLT_PORT_RANGE)) + .setPluginProviders(cfgPlugin); + + if (!cfg.isClientMode()) { + cfg.setDataStorageConfiguration(new DataStorageConfiguration() + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setPersistenceEnabled(true))); + + cfg.getDataStorageConfiguration() + .setWalForceArchiveTimeout(5_000) + .setChangeDataCaptureEnabled(true); + + cfg.setConsistentId(igniteInstanceName); + } + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + cleanPersistenceDir(); + + kafka.start(); + + if (props == null) { + props = new Properties(); + + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka.getBootstrapServers()); + + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class.getName()); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, JavaObjectSerializer.class.getName()); + + props.put(ConsumerConfig.GROUP_ID_CONFIG, "kafka-to-ignite-applier"); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class.getName()); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.put(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, 10_000); + } + + createTopic(DFLT_TOPIC, DFLT_PARTS, props); + createTopic(SRC_DEST_TOPIC, DFLT_PARTS, props); + createTopic(DEST_SRC_TOPIC, DFLT_PARTS, props); + + srcCluster = new IgniteEx[] { + startGrid(1), + startGrid(2) + }; + + srcClusterCliCfg = optimize(getConfiguration("src-cluster-client").setClientMode(true)); + + srcCluster[0].cluster().state(ACTIVE); + srcCluster[0].cluster().tag("source"); + + discoPort += DFLT_PORT_RANGE + 1; + commPort += DFLT_PORT_RANGE + 1; + clusterId = DEST_CLUSTER_ID; + + destCluster = new IgniteEx[] { + startGrid(4), + startGrid(5) + }; + + destClusterCliCfg = optimize(getConfiguration("dest-cluster-client").setClientMode(true)); + + assertFalse("source".equals(destCluster[0].cluster().tag())); + + destCluster[0].cluster().state(ACTIVE); + destCluster[0].cluster().tag("destination"); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + props = null; + + stopAllGrids(); + + cleanPersistenceDir(); + + kafka.stop(); + } + + /** */ + @Test + public void testActivePassiveReplication() throws Exception { + IgniteInternalFuture<?> fut1 = igniteToKafka(srcCluster[0], DFLT_TOPIC, AP_CACHE); + IgniteInternalFuture<?> fut2 = igniteToKafka(srcCluster[1], DFLT_TOPIC, AP_CACHE); + + try { + IgniteCache<Integer, Data> destCache = destCluster[0].createCache(AP_CACHE); + + destCache.put(1, new Data(null, 0, 1, REQUEST_ID.incrementAndGet())); + destCache.remove(1); + + runAsync(generateData("cache-1", srcCluster[srcCluster.length - 1], IntStream.range(0, KEYS_CNT), 1)); + runAsync(generateData(AP_CACHE, srcCluster[srcCluster.length - 1], IntStream.range(0, KEYS_CNT), 1)); + + KafkaToIgniteCdcStreamerConfiguration cfg = new KafkaToIgniteCdcStreamerConfiguration(); + + cfg.setCacheNames(Collections.singletonList(AP_CACHE)); + + IgniteInternalFuture<?> k2iFut = + runAsync(new KafkaToIgniteCdcStreamer(destClusterCliCfg, props, cfg)); + + try { + IgniteCache<Integer, Data> srcCache = srcCluster[srcCluster.length - 1].getOrCreateCache(AP_CACHE); + + waitForSameData(srcCache, destCache, KEYS_CNT, BOTH_EXISTS, 1, fut1, fut2, k2iFut); + + IntStream.range(0, KEYS_CNT).forEach(srcCache::remove); + + waitForSameData(srcCache, destCache, KEYS_CNT, BOTH_REMOVED, 1, fut1, fut2, k2iFut); + } + finally { + k2iFut.cancel(); + } + } + finally { + fut1.cancel(); + fut2.cancel(); + } + } + + /** */ + @Test + public void testActiveActiveReplication() throws Exception { + IgniteCache<Integer, Data> srcCache = srcCluster[0].getOrCreateCache(ACTIVE_ACTIVE_CACHE); + IgniteCache<Integer, Data> destCache = destCluster[0].getOrCreateCache(ACTIVE_ACTIVE_CACHE); + + runAsync(generateData(ACTIVE_ACTIVE_CACHE, srcCluster[srcCluster.length - 1], + IntStream.range(0, KEYS_CNT).filter(i -> i % 2 == 0), 1)); + runAsync(generateData(ACTIVE_ACTIVE_CACHE, destCluster[destCluster.length - 1], + IntStream.range(0, KEYS_CNT).filter(i -> i % 2 != 0), 1)); + + IgniteInternalFuture<?> cdcSrcFut1 = igniteToKafka(srcCluster[0], SRC_DEST_TOPIC, ACTIVE_ACTIVE_CACHE); + IgniteInternalFuture<?> cdcSrcFut2 = igniteToKafka(srcCluster[1], SRC_DEST_TOPIC, ACTIVE_ACTIVE_CACHE); + IgniteInternalFuture<?> cdcDestFut1 = igniteToKafka(destCluster[0], DEST_SRC_TOPIC, ACTIVE_ACTIVE_CACHE); + IgniteInternalFuture<?> cdcDestFut2 = igniteToKafka(destCluster[1], DEST_SRC_TOPIC, ACTIVE_ACTIVE_CACHE); + + try { + KafkaToIgniteCdcStreamerConfiguration cfg1 = new KafkaToIgniteCdcStreamerConfiguration(); + KafkaToIgniteCdcStreamerConfiguration cfg2 = new KafkaToIgniteCdcStreamerConfiguration(); + + cfg1.setTopic(SRC_DEST_TOPIC); + cfg1.setCacheNames(Collections.singletonList(ACTIVE_ACTIVE_CACHE)); + + cfg2.setTopic(DEST_SRC_TOPIC); + cfg2.setCacheNames(Collections.singletonList(ACTIVE_ACTIVE_CACHE)); + + IgniteInternalFuture<?> k2iFut1 = runAsync(new KafkaToIgniteCdcStreamer(destClusterCliCfg, props, cfg1)); + IgniteInternalFuture<?> k2iFut2 = runAsync(new KafkaToIgniteCdcStreamer(srcClusterCliCfg, props, cfg2)); + + try { + waitForSameData(srcCache, destCache, KEYS_CNT, BOTH_EXISTS, 1, + cdcDestFut1, cdcDestFut2, cdcDestFut1, cdcDestFut2, k2iFut1, k2iFut2); + + for (int i = 0; i < KEYS_CNT; i++) { + srcCache.put(i, generateSingleData(2)); + destCache.put(i, generateSingleData(2)); + } + + waitForSameData(srcCache, destCache, KEYS_CNT, ANY_SAME_STATE, 2, + cdcDestFut1, cdcDestFut2, cdcDestFut1, cdcDestFut2, k2iFut1, k2iFut2); + } + finally { + k2iFut1.cancel(); + k2iFut2.cancel(); + } + } + finally { + cdcSrcFut1.cancel(); + cdcSrcFut2.cancel(); + cdcDestFut1.cancel(); + cdcDestFut2.cancel(); + } + } + + /** */ + public void waitForSameData( + IgniteCache<Integer, Data> src, + IgniteCache<Integer, Data> dest, + int keysCnt, + int keysState, + int iter, + IgniteInternalFuture<?>...futs + ) throws IgniteInterruptedCheckedException { + assertTrue(waitForCondition(() -> { + for (int i = 0; i < keysCnt; i++) { + if (keysState == BOTH_EXISTS) { + if (!src.containsKey(i) || !dest.containsKey(i)) + return checkFuts(false, futs); + } + else if (keysState == BOTH_REMOVED) { + if (src.containsKey(i) || dest.containsKey(i)) + return checkFuts(false, futs); + + continue; + } + else if (keysState == ANY_SAME_STATE) { + if (src.containsKey(i) != dest.containsKey(i)) + return checkFuts(false, futs); + + if (!src.containsKey(i)) + continue; + } + else + throw new IllegalArgumentException(keysState + " not supported."); + + Data data = dest.get(i); + + if (!data.equals(src.get(i))) + return checkFuts(false, futs); + + checkCRC(data, iter); + } + + return checkFuts(true, futs); + }, getTestTimeout())); + } + + /** */ + private boolean checkFuts(boolean res, IgniteInternalFuture<?>...futs) { + for (IgniteInternalFuture<?> fut : futs) + assertFalse(fut.isDone()); + + return res; + } + + /** */ + public static void checkCRC(Data data, int iter) { + assertEquals(iter, data.iter); + + crc.get().reset(); + crc.get().update(ByteBuffer.wrap(data.payload), data.payload.length); + + assertEquals(crc.get().getValue(), data.crc); + } + + /** + * @param ign Ignite instance to watch for. + * @param topic Kafka topic name. + * @param caches Caches names to stream to kafka. + * @return Future for Change Data Capture application. + */ + private IgniteInternalFuture<?> igniteToKafka(IgniteEx ign, String topic, String...caches) { Review comment: String...caches feature newer used. So, seems we able to get gid of `new HashSet<>(Arrays.asList(caches))` anyway, whitespace missed. -- 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]
