Mmuzaf commented on a change in pull request #8909: URL: https://github.com/apache/ignite/pull/8909#discussion_r598164767
########## File path: modules/core/src/main/java/org/apache/ignite/cdc/ChangeEvent.java ########## @@ -0,0 +1,150 @@ +/* + * 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.Serializable; +import org.apache.ignite.internal.cdc.IgniteCDC; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteExperimental; + +/** + * Event of single entry change. + * Instance presents new value of modified entry. + * + * @param <K> Key type. + * @param <V> Value type. + * @see IgniteCDC + * @see CDCConsumer + */ +@IgniteExperimental +public class ChangeEvent<K, V> implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Key. */ + @GridToStringInclude(sensitive = true) Review comment: This is a part of the internal API, probably we should remove it. ########## File path: modules/core/src/main/java/org/apache/ignite/cdc/CDCConsumer.java ########## @@ -0,0 +1,78 @@ +/* + * 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.Iterator; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.cdc.IgniteCDC; +import org.apache.ignite.lang.IgniteExperimental; + +/** + * Consumer of WAL data change events. + * This consumer will receive event of data changes during {@link IgniteCDC} application invocation. + * Lifecycle of consumer is the following: + * <ul> + * <li>Start of the consumer {@link #start(IgniteConfiguration, IgniteLogger)}</li> + * <li>Notification of the consumer by the {@link #onChange(Iterator)} call.</li> + * <li>Stop of the consumer {@link #stop()}</li> + * </ul> + * + * Note, consumption of the {@link ChangeEvent} will started from the last saved offset. + * Offset of consumptions is saved on the disk every time {@link #onChange(Iterator)} returns {@code true}. + * + * @see IgniteCDC + * @see ChangeEvent + * @see ChangeEventOrder + */ +@IgniteExperimental +public interface CDCConsumer<K, V> { Review comment: I suggest keeping the full name like `CaptureDataChangeConsumer` because users may not be familiar with such abbreviations during searching some examples. ########## File path: modules/core/src/main/java/org/apache/ignite/internal/cdc/CDCConsumerState.java ########## @@ -0,0 +1,113 @@ +/* + * 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.internal.cdc; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.Iterator; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cdc.CDCConsumer; +import org.apache.ignite.cdc.ChangeEvent; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; +import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX; +import static org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer.POINTER_SIZE; + +/** + * CDC Consumer state. + * + * Each time {@link CDCConsumer#onChange(Iterator)} returns {@code true} current offset in WAL segment saved to file. + * This allows to the {@link CDCConsumer} to continue consumption of the {@link ChangeEvent} from the last saved offset in case of fail or restart. + * + * @see CDCConsumer#onChange(Iterator) + * @see IgniteCDC + */ +public class CDCConsumerState { + /** State file. */ + private final Path state; + + /** Temp state file. */ + private final Path tmp; + + /** + * @param stateDir State directory. + * @param consumerId Consumer ID. + */ + public CDCConsumerState(Path stateDir, String consumerId) { + String fileName = "state-" + U.maskForFileName(consumerId) + FILE_SUFFIX; + + state = stateDir.resolve(fileName); + tmp = stateDir.resolve(fileName + TMP_SUFFIX); + } + + /** + * Saves state to file. + * @param ptr WAL pointer. + */ + public void save(WALPointer ptr) throws IOException { + ByteBuffer buf = ByteBuffer.allocate(POINTER_SIZE); + + buf.putLong(ptr.index()); + buf.putInt(ptr.fileOffset()); + buf.putInt(ptr.length()); + buf.flip(); + + try (FileChannel ch = FileChannel.open(tmp, StandardOpenOption.CREATE, StandardOpenOption.WRITE)) { Review comment: Opening the `FileChannel` each time we are trying to save a consumed batch of events my be a heavy operation. Would it be better to keep it opened while the segment still processing? ########## File path: modules/core/src/main/java/org/apache/ignite/cdc/ChangeEventOrder.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.io.Serializable; +import java.util.Objects; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.version.GridCacheVersion.DR_ID_MASK; Review comment: Probably should make this constants `public` too. ########## File path: modules/core/src/main/java/org/apache/ignite/internal/cdc/IgniteCDC.java ########## @@ -0,0 +1,505 @@ +/* + * 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.internal.cdc; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.function.Predicate; +import java.util.stream.Stream; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cdc.CDCConsumer; +import org.apache.ignite.cdc.ChangeEvent; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgnitionEx; +import org.apache.ignite.internal.MarshallerContextImpl; +import org.apache.ignite.internal.pagemem.wal.WALIterator; +import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; +import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory; +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.ignite.lang.IgniteBiTuple; + +import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_CDC_PATH; +import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.NODE_PATTERN; +import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.UUID_STR_PATTERN; +import static org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.WAL_NAME_PATTERN; + +/** + * CDC(Capture Data Change) application. + * Application run independently of Ignite node process and provide ability for the {@link CDCConsumer} to consume events({@link ChangeEvent}) from WAL segments. + * User should responsible {@link CDCConsumer} implementation with custom consumption logic. + * + * Ignite node should be explicitly configured for using {@link IgniteCDC}. + * <ol> + * <li>Set {@link DataStorageConfiguration#setCdcEnabled(boolean)} to true.</li> + * <li>Optional: Set {@link DataStorageConfiguration#setCdcPath(String)} to path to the directory to store WAL setgments for CDC.</li> + * <li>Optional: Set {@link DataStorageConfiguration#setWalForceArchiveTimeout(long)} to configure timeout for force WAL rollover, + * so new events will be available for consumptions with the predicted time.</li> + * </ol> + * + * When {@link DataStorageConfiguration#getCdcPath()} is true then Ignite node on each WAL segment rollover creates hard link + * to archive WAL segment in {@link DataStorageConfiguration#getCdcPath()} directory. + * {@link IgniteCDC} application takes segment file and consumes events from it. After successfull consumption (see {@link CDCConsumer#onChange(Iterator)}) + * WAL segement will be deleted from directory. + * + * Several Ignite nodes can be started on the same host. + * In that case you can specify {@link IgniteConfiguration#getConsistentId()} in provided {@link IgniteConfiguration} or set system properties to specify node to use. + * <ul> + * <li>{@link #IGNITE_CDC_CONSISTENT_ID} - property to specify node consistent id.</li> + * <li>{@link #IGNITE_CDC_NODE_IDX} - property to specify node index.</li> + * </ul> + * + * Application works as follows: + * <ol> + * <li>Search node work directory based on provided {@link IgniteConfiguration} and system properties.</li> + * <li>Await for creation of CDC directory if it not exists.</li> + * <li>Acquire file lock to ensure exclusive consumption.</li> + * <li>Loads state of consumption if it exists.</li> + * <li>Infinetely wait for new available segement and process it.</li> + * </ol> + * + * @see DataStorageConfiguration#setCdcEnabled(boolean) + * @see DataStorageConfiguration#setCdcPath(String) + * @see DataStorageConfiguration#setWalForceArchiveTimeout(long) + * @see CommandLineStartup + * @see CDCConsumer + * @see DataStorageConfiguration#DFLT_CDC_PATH + */ +public class IgniteCDC implements Runnable { Review comment: Should it be public also? Do we need abbreviations in class naming? ########## File path: bin/ignite-cdc.sh ########## @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +# +# 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. +# + +export MAIN_CLASS="org.apache.ignite.cdc.CommandLineStartup" + +if [ "${IGNITE_HOME:-}" = "" ]; + then IGNITE_HOME_TMP="$(dirname "$(cd "$(dirname "$0")"; "pwd")")"; + else IGNITE_HOME_TMP=${IGNITE_HOME}; +fi + +${IGNITE_HOME_TMP}/bin/ignite.sh "$@" Review comment: Not sure we should share the same startup script for the CDC application. It may have some custom JVM options which may prevent using CDC application, e.g. having different defaults for the size of heap may be a good idea. ########## File path: modules/core/src/main/java/org/apache/ignite/cdc/ChangeEventOrder.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.io.Serializable; +import java.util.Objects; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.version.GridCacheVersion.DR_ID_MASK; +import static org.apache.ignite.internal.processors.cache.version.GridCacheVersion.DR_ID_SHIFT; + +/** + * Entry event order. + * Two concurrent updates of the same entry can be ordered based on {@link ChangeEventOrder} comparsion. + * Greater value means that event occurs later. + */ +public class ChangeEventOrder implements Comparable<ChangeEventOrder>, Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** Topology version. */ + private final int topVer; + + /** Node order (used as global order) and DR ID. */ + private final int nodeOrderDrId; + + /** Order. */ + private final long order; + + /** Replica version. */ + private @Nullable ChangeEventOrder otherDcOrder; + + /** + * @param topVer Topology version plus number of seconds from the start time of the first grid node. + * @param nodeOrderDrId Node order and DR ID. + * @param order Version order. + */ + public ChangeEventOrder(int topVer, int nodeOrderDrId, long order) { + this.topVer = topVer; + this.nodeOrderDrId = nodeOrderDrId; + this.order = order; + } + + /** @return topVer Topology version plus number of seconds from the start time of the first grid node. */ + public int topVer() { + return topVer; + } + + /** @return nodeOrderDrId Node order and DR ID. */ + public int nodeOrderDrId() { + return nodeOrderDrId; + } + + /** + * @return DR mask. + */ + public byte dataCenterId() { + return (byte)((nodeOrderDrId >> DR_ID_SHIFT) & DR_ID_MASK); + } + + /** @return order Version order. */ + public long order() { + return order; + } + + /** @param replicaVer Replication version. */ + public void otherDcOrder(ChangeEventOrder replicaVer) { + this.otherDcOrder = replicaVer; + } + + /** @return Replication version. */ + public ChangeEventOrder otherDcOrder() { + return otherDcOrder; + } + + /** {@inheritDoc} */ + @Override public int compareTo(@NotNull ChangeEventOrder other) { + int res = Integer.compare(topVer, other.topVer); + + if (res != 0) + return res; + + res = Long.compare(order, other.order); + + if (res != 0) + return res; + + return Integer.compare(nodeOrderDrId, other.nodeOrderDrId); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + ChangeEventOrder order1 = (ChangeEventOrder)o; + return topVer == order1.topVer && nodeOrderDrId == order1.nodeOrderDrId && order == order1.order; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(topVer, nodeOrderDrId, order); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ChangeEventOrder.class, this); Review comment: `S.toString()` is a part of the internal API. ########## File path: modules/core/src/main/java/org/apache/ignite/cdc/CDCConsumer.java ########## @@ -0,0 +1,78 @@ +/* + * 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.Iterator; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.cdc.IgniteCDC; +import org.apache.ignite.lang.IgniteExperimental; + +/** + * Consumer of WAL data change events. + * This consumer will receive event of data changes during {@link IgniteCDC} application invocation. + * Lifecycle of consumer is the following: + * <ul> + * <li>Start of the consumer {@link #start(IgniteConfiguration, IgniteLogger)}</li> + * <li>Notification of the consumer by the {@link #onChange(Iterator)} call.</li> + * <li>Stop of the consumer {@link #stop()}</li> + * </ul> + * + * Note, consumption of the {@link ChangeEvent} will started from the last saved offset. + * Offset of consumptions is saved on the disk every time {@link #onChange(Iterator)} returns {@code true}. + * + * @see IgniteCDC + * @see ChangeEvent + * @see ChangeEventOrder + */ +@IgniteExperimental +public interface CDCConsumer<K, V> { + /** + * @return Consumer ID. + */ + String id(); + + /** + * Starts the consumer. + * + * @param configuration Ignite configuration. + * @param log Logger. + */ + void start(IgniteConfiguration configuration, IgniteLogger log); + + /** + * @return {@code True} if entry key and value should be keeped in binary format. + */ + boolean keepBinary(); Review comment: Do we need this property changing at runtime and/or during setting the configuration? It seems it will always be `true` by default. -- 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]
