[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1081080438 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/ConcurrentTxsConsistentCutTest.java: ## @@ -0,0 +1,269 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.function.BiFunction; +import java.util.function.Supplier; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotSelfTest.snp; + +/** Load Ignite with transactions and starts Consistent Cut concurrently. */ +@RunWith(Parameterized.class) +public class ConcurrentTxsConsistentCutTest extends AbstractConsistentCutTest { +/** Amount of Consistent Cuts to await. */ +private static final int CUTS = 20; + +/** */ +private static final String CACHE2 = "CACHE2"; + +/** */ +private static final Random RND = new Random(); + +/** */ +private final AtomicInteger txCnt = new AtomicInteger(); + +/** Notifies data loader to stop preparing new transactions. */ +private volatile CountDownLatch stopLoadLatch; + +/** Number of server nodes. */ +@Parameterized.Parameter +public int nodes; + +/** Number of backups. */ +@Parameterized.Parameter(1) +public int backups; + +/** */ +@Parameterized.Parameter(2) +public boolean withNearCache; + +/** */ +@Parameterized.Parameters(name = "nodes={0}, backups={1}, withNearCache={2}") +public static List params() { +int[][] nodesAndBackupsCnt = new int[][] { +new int[] {3, 0}, +new int[] {2, 1}, +new int[] {3, 2} +}; + +List params = new ArrayList<>(); + +for (int[] nb: nodesAndBackupsCnt) { +for (boolean near: new boolean[] {false, true}) +params.add(new Object[] {nb[0], nb[1], near}); +} + +return params; +} + +/** {@inheritDoc} */ +@Override protected int nodes() { +return nodes; +} + +/** {@inheritDoc} */ +@Override protected int backups() { +return backups; +} + +/** {@inheritDoc} */ +@Override protected IgniteConfiguration getConfiguration(String instanceName) throws Exception { +IgniteConfiguration cfg = super.getConfiguration(instanceName); + +CacheConfiguration ccfg1 = cacheConfiguration(CACHE); + +if (withNearCache) +ccfg1.setNearConfiguration(new NearCacheConfiguration<>()); + +// Set less partitions to handle default open files limit in OS. +CacheConfiguration ccfg2 = cacheConfiguration(CACHE2) +.setAffinity(new RendezvousAffinityFunction(false, 100)); + +cfg.setCacheConfiguration(ccfg1, ccfg2); + +return cfg; +} + +/** */ +@Test +public void noLoadTest() throws Exception { +testConcurrentTransactionsAndCuts(() -> false); +} + +/** */ +@Test +public void concurrentLoadTransactionsTest() throws Exception { +testConcurrentTransactionsAndCuts(() -> explicitTransaction((g, tx) -> { +int cnt = 1 + RND.nextInt(nodes()); + +for (int j = 0; j < cnt; j++) { +IgniteCache cache = g.cache(CACHE); + +cache.put(RND.nextInt(), RND.nextInt()); +
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1081076810 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/AbstractConsistentCutTest.java: ## @@ -0,0 +1,296 @@ +/* + * 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.processors.cache.consistentcut; + +import java.io.File; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cluster.ClusterState; +import org.apache.ignite.configuration.CacheConfiguration; +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.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.pagemem.wal.WALIterator; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.DataRecord; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +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.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.CONSISTENT_CUT_FINISH_RECORD; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.CONSISTENT_CUT_START_RECORD; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2; +import static org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotSelfTest.snp; + +/** Base class for testing Consistency Cut algorithm. */ +public abstract class AbstractConsistentCutTest extends GridCommonAbstractTest { +/** */ +protected static final String CACHE = "CACHE"; + +/** */ +protected static final String SNP = "base"; + +/** {@inheritDoc} */ +@Override protected IgniteConfiguration getConfiguration(String instanceName) throws Exception { +IgniteConfiguration cfg = super.getConfiguration(instanceName); + +cfg.setDataStorageConfiguration(new DataStorageConfiguration() +.setWalCompactionEnabled(true) +.setDefaultDataRegionConfiguration(new DataRegionConfiguration() +.setName("consistent-cut-persist") +.setPersistenceEnabled(true))); + +cfg.setCacheConfiguration(cacheConfiguration(CACHE)); + +cfg.setConsistentId(instanceName); + +return cfg; +} + +/** {@inheritDoc} */ +@Override protected void beforeTest() throws Exception { +cleanPersistenceDir(); + +startGrids(nodes()); + +grid(0).cluster().state(ClusterState.ACTIVE); + +startClientGrid(nodes()); + +snp(grid(0)).createSnapshot(SNP).get(); +} + +/** {@inheritDoc} */ +@Override protected void afterTest() throws Exception { +stopAllGrids(); + +cleanPersistenceDir(); +} + +/** */ +protected CacheConfiguration cacheConfiguration(String cacheName) { +return new CacheConfiguration()
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1081075077 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/AbstractConsistentCutTest.java: ## @@ -0,0 +1,296 @@ +/* + * 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.processors.cache.consistentcut; + +import java.io.File; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cluster.ClusterState; +import org.apache.ignite.configuration.CacheConfiguration; +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.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.pagemem.wal.WALIterator; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.DataRecord; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +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.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.CONSISTENT_CUT_FINISH_RECORD; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.CONSISTENT_CUT_START_RECORD; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2; +import static org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotSelfTest.snp; + +/** Base class for testing Consistency Cut algorithm. */ +public abstract class AbstractConsistentCutTest extends GridCommonAbstractTest { +/** */ +protected static final String CACHE = "CACHE"; + +/** */ +protected static final String SNP = "base"; + +/** {@inheritDoc} */ +@Override protected IgniteConfiguration getConfiguration(String instanceName) throws Exception { +IgniteConfiguration cfg = super.getConfiguration(instanceName); + +cfg.setDataStorageConfiguration(new DataStorageConfiguration() +.setWalCompactionEnabled(true) +.setDefaultDataRegionConfiguration(new DataRegionConfiguration() +.setName("consistent-cut-persist") +.setPersistenceEnabled(true))); + +cfg.setCacheConfiguration(cacheConfiguration(CACHE)); + +cfg.setConsistentId(instanceName); + +return cfg; +} + +/** {@inheritDoc} */ +@Override protected void beforeTest() throws Exception { +cleanPersistenceDir(); + +startGrids(nodes()); + +grid(0).cluster().state(ClusterState.ACTIVE); + +startClientGrid(nodes()); + +snp(grid(0)).createSnapshot(SNP).get(); +} + +/** {@inheritDoc} */ +@Override protected void afterTest() throws Exception { +stopAllGrids(); + +cleanPersistenceDir(); +} + +/** */ +protected CacheConfiguration cacheConfiguration(String cacheName) { +return new CacheConfiguration()
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1081037421 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/ConsistentCutMarkWalFuture.java: ## @@ -0,0 +1,186 @@ +/* + * 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.processors.cache.persistence.snapshot; + +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; + +/** + * Consistent Cut is a distributed algorithm that defines two set of transactions - BEFORE and AFTER cut - on baseline nodes. + * It guarantees that every transaction was included into BEFORE on one node also included into the BEFORE on every other node + * participated in the transaction. It means that Ignite nodes can safely recover themselves to the consistent BEFORE + * state without any coordination with each other. + */ +class ConsistentCutMarkWalFuture extends GridFutureAdapter { +/** Cache context. */ +private final GridCacheSharedContext cctx; + +/** Consistent Cut ID. */ +private final UUID id; + +/** Logger. */ +private final IgniteLogger log; + +/** Set of checked transactions belong to the BEFORE set. */ +@GridToStringInclude +private final Set included = ConcurrentHashMap.newKeySet(); + +/** Set of checked transactions belong to the AFTER set. */ Review Comment: This comment must be updated. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1081037100 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/ConsistentCutMarkWalFuture.java: ## @@ -0,0 +1,186 @@ +/* + * 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.processors.cache.persistence.snapshot; + +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; + +/** + * Consistent Cut is a distributed algorithm that defines two set of transactions - BEFORE and AFTER cut - on baseline nodes. Review Comment: This comment must be updated. ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/ConsistentCutMarkWalFuture.java: ## @@ -0,0 +1,186 @@ +/* + * 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.processors.cache.persistence.snapshot; + +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apa
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1081036626 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,101 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * This record is written to WAL after Consistent Cut finished on a baseline node. + * + * During recovery node must apply: + * 1. Transactions committed before {@link ConsistentCutStartRecord} except those contained in {@link #excluded()}. + * 2. Transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and contained in {@link #included()}. + */ +public class ConsistentCutFinishRecord extends WALRecord { +/** Consistent Cut ID. */ +@GridToStringInclude +private final UUID id; + +/** + * Set of transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} Review Comment: This comment must be updated. ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,101 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * This record is written to WAL after Consistent Cut finished on a baseline node. + * + * During recovery node must apply: + * 1. Transactions committed before {@link ConsistentCutStartRecord} except those contained in {@link #excluded()}. + * 2. Transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and contained in {@link #included()}. + */ +public class ConsistentCutFinishRecord extends WALRecord { +/** Consistent Cut ID. */ +@GridToStringInclude +private final UUID id; + +/** + * Set of transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + * to include to the BEFORE side of Consistent Cut. + */ +@GridToStringInclude +private final Set included; + +/** + * Set of transactions committed before {@link ConsistentCutStartRecord} to exclude from the AFTER side of Consistent Cut. Review Comment: This comment must be updated. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071152344 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/AbstractConsistentCutTest.java: ## @@ -0,0 +1,301 @@ +/* + * 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.processors.cache.consistentcut; + +import java.io.File; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cluster.ClusterState; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.pagemem.wal.WALIterator; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.DataRecord; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +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.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.CONSISTENT_CUT_FINISH_RECORD; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.CONSISTENT_CUT_START_RECORD; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2; +import static org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotSelfTest.snp; + +/** Base class for testing Consistency Cut algorithm. */ +public abstract class AbstractConsistentCutTest extends GridCommonAbstractTest { +/** */ +protected static final String CACHE = "CACHE"; + +/** */ +protected static final String SNP = "base"; + +/** {@inheritDoc} */ +@Override protected IgniteConfiguration getConfiguration(String instanceName) throws Exception { +IgniteConfiguration cfg = super.getConfiguration(instanceName); + +cfg.setDataStorageConfiguration(new DataStorageConfiguration() +.setWalCompactionEnabled(true) +.setDefaultDataRegionConfiguration(new DataRegionConfiguration() +.setName("consistent-cut-persist") +.setPersistenceEnabled(true))); + +CacheConfiguration ccfg = new CacheConfiguration() +.setName(CACHE) +.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) +.setBackups(backups()); + +if (withNearCache()) +ccfg.setNearConfiguration(new NearCacheConfiguration<>()); + +cfg.setCacheConfiguration(ccfg); + +cfg.setConsistentId(instanceName); + +return cfg; +} + +/** {@inheritDoc} */ +@Override protected void beforeTest() throws Exception { +cleanPersistenceDir(); + +startGrids(nodes()); + +grid(0).cluster().state(ClusterState.ACTIVE); + +startClientGrid(nodes()); + +snp(grid(0)).creat
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071136370 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/ConcurrentTxsConsistentCutTest.java: ## @@ -0,0 +1,194 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotSelfTest.snp; + +/** Load Ignite with transactions and starts Consistent Cut concurrently. */ +@RunWith(Parameterized.class) +public class ConcurrentTxsConsistentCutTest extends AbstractConsistentCutTest { +/** Amount of Consistent Cuts to await. */ +private static final int CUTS = 20; + +/** */ +private static final Random RND = new Random(); + +/** */ +private final AtomicInteger txCnt = new AtomicInteger(); + +/** Notifies data loader to stop preparing new transactions. */ +private volatile CountDownLatch stopLoadLatch; + +/** Number of server nodes. */ +@Parameterized.Parameter +public int nodes; + +/** Number of backups. */ +@Parameterized.Parameter(1) +public int backups; + +/** */ +@Parameterized.Parameter(2) +public boolean withNearCache; + +/** */ +@Parameterized.Parameters(name = "nodes={0}, backups={1}, withNearCache={2}") +public static List params() { +List> nodesAndBackups = F.asList( +new T2<>(3, 0), +new T2<>(2, 1), +new T2<>(3, 2)); + +List params = new ArrayList<>(); + +for (T2 nb: nodesAndBackups) { +for (boolean near: new boolean[] {false, true}) +params.add(new Object[] {nb.get1(), nb.get2(), near}); +} + +return params; +} + +/** {@inheritDoc} */ +@Override protected int nodes() { +return nodes; +} + +/** {@inheritDoc} */ +@Override protected int backups() { +return backups; +} + +/** {@inheritDoc} */ +@Override protected boolean withNearCache() { +return withNearCache; +} + +/** */ +@Test +public void noLoadTest() throws Exception { +testConcurrentTransactionsAndCuts(() -> {}, false); +} + +/** */ +@Test +public void concurrentLoadTransactionsTest() throws Exception { +testConcurrentTransactionsAndCuts(() -> { +// +1 - client node. +int n = RND.nextInt(nodes() + 1); + +Ignite g = grid(n); + +try (Transaction tx = g.transactions().txStart()) { +int cnt = 1 + RND.nextInt(nodes()); + +for (int j = 0; j < cnt; j++) { +IgniteCache cache = g.cache(CACHE); + +cache.put(RND.nextInt(), RND.nextInt()); +} + +tx.commit(); +} +}, true); +} + +/** */ +@Test +public void concurrentLoadImplicitTransactionsTest() throws Exception { +testConcurrentTransactionsAndCuts(() -> { +// +1 - client node. +int n = RND.nextInt(nodes() + 1); + +IgniteCache cache = grid(n).cache(CACHE); + +cache.put(RND.nextInt(), RND.nextInt()); +}, true); +} + +/** */ +@Test +public void concurrentLoadImplicitTransactionsAndExplicitLocksTest() throws Exception { +testConcurrentTransactionsAndCuts(() -> { +// +1 - client node. +int n = RND.nextInt(nodes() + 1
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071109454 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/ConcurrentTxsConsistentCutTest.java: ## @@ -0,0 +1,194 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotSelfTest.snp; + +/** Load Ignite with transactions and starts Consistent Cut concurrently. */ +@RunWith(Parameterized.class) +public class ConcurrentTxsConsistentCutTest extends AbstractConsistentCutTest { +/** Amount of Consistent Cuts to await. */ +private static final int CUTS = 20; + +/** */ +private static final Random RND = new Random(); + +/** */ +private final AtomicInteger txCnt = new AtomicInteger(); + +/** Notifies data loader to stop preparing new transactions. */ +private volatile CountDownLatch stopLoadLatch; + +/** Number of server nodes. */ +@Parameterized.Parameter +public int nodes; + +/** Number of backups. */ +@Parameterized.Parameter(1) +public int backups; + +/** */ +@Parameterized.Parameter(2) +public boolean withNearCache; + +/** */ +@Parameterized.Parameters(name = "nodes={0}, backups={1}, withNearCache={2}") +public static List params() { +List> nodesAndBackups = F.asList( +new T2<>(3, 0), +new T2<>(2, 1), +new T2<>(3, 2)); + +List params = new ArrayList<>(); + +for (T2 nb: nodesAndBackups) { +for (boolean near: new boolean[] {false, true}) +params.add(new Object[] {nb.get1(), nb.get2(), near}); +} + +return params; +} + +/** {@inheritDoc} */ +@Override protected int nodes() { +return nodes; +} + +/** {@inheritDoc} */ +@Override protected int backups() { +return backups; +} + +/** {@inheritDoc} */ +@Override protected boolean withNearCache() { +return withNearCache; +} + +/** */ +@Test +public void noLoadTest() throws Exception { +testConcurrentTransactionsAndCuts(() -> {}, false); +} + +/** */ +@Test +public void concurrentLoadTransactionsTest() throws Exception { +testConcurrentTransactionsAndCuts(() -> { +// +1 - client node. +int n = RND.nextInt(nodes() + 1); + +Ignite g = grid(n); + +try (Transaction tx = g.transactions().txStart()) { +int cnt = 1 + RND.nextInt(nodes()); + +for (int j = 0; j < cnt; j++) { +IgniteCache cache = g.cache(CACHE); + +cache.put(RND.nextInt(), RND.nextInt()); +} + +tx.commit(); Review Comment: We need randomly commit or rollback transactions here. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071107606 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/ConcurrentTxsConsistentCutTest.java: ## @@ -0,0 +1,194 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotSelfTest.snp; + +/** Load Ignite with transactions and starts Consistent Cut concurrently. */ +@RunWith(Parameterized.class) +public class ConcurrentTxsConsistentCutTest extends AbstractConsistentCutTest { +/** Amount of Consistent Cuts to await. */ +private static final int CUTS = 20; + +/** */ +private static final Random RND = new Random(); + +/** */ +private final AtomicInteger txCnt = new AtomicInteger(); + +/** Notifies data loader to stop preparing new transactions. */ +private volatile CountDownLatch stopLoadLatch; + +/** Number of server nodes. */ +@Parameterized.Parameter +public int nodes; + +/** Number of backups. */ +@Parameterized.Parameter(1) +public int backups; + +/** */ +@Parameterized.Parameter(2) +public boolean withNearCache; + +/** */ +@Parameterized.Parameters(name = "nodes={0}, backups={1}, withNearCache={2}") +public static List params() { +List> nodesAndBackups = F.asList( +new T2<>(3, 0), +new T2<>(2, 1), +new T2<>(3, 2)); + +List params = new ArrayList<>(); + +for (T2 nb: nodesAndBackups) { +for (boolean near: new boolean[] {false, true}) +params.add(new Object[] {nb.get1(), nb.get2(), near}); +} + +return params; +} + +/** {@inheritDoc} */ +@Override protected int nodes() { +return nodes; +} + +/** {@inheritDoc} */ +@Override protected int backups() { +return backups; +} + +/** {@inheritDoc} */ +@Override protected boolean withNearCache() { +return withNearCache; +} + +/** */ +@Test +public void noLoadTest() throws Exception { +testConcurrentTransactionsAndCuts(() -> {}, false); +} + +/** */ +@Test +public void concurrentLoadTransactionsTest() throws Exception { +testConcurrentTransactionsAndCuts(() -> { +// +1 - client node. +int n = RND.nextInt(nodes() + 1); + +Ignite g = grid(n); + +try (Transaction tx = g.transactions().txStart()) { +int cnt = 1 + RND.nextInt(nodes()); + +for (int j = 0; j < cnt; j++) { +IgniteCache cache = g.cache(CACHE); Review Comment: We need to test several cache transactions. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071100462 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/ConcurrentTxsConsistentCutTest.java: ## @@ -0,0 +1,194 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotSelfTest.snp; + +/** Load Ignite with transactions and starts Consistent Cut concurrently. */ +@RunWith(Parameterized.class) +public class ConcurrentTxsConsistentCutTest extends AbstractConsistentCutTest { +/** Amount of Consistent Cuts to await. */ +private static final int CUTS = 20; + +/** */ +private static final Random RND = new Random(); + +/** */ +private final AtomicInteger txCnt = new AtomicInteger(); + +/** Notifies data loader to stop preparing new transactions. */ +private volatile CountDownLatch stopLoadLatch; + +/** Number of server nodes. */ +@Parameterized.Parameter +public int nodes; + +/** Number of backups. */ +@Parameterized.Parameter(1) +public int backups; + +/** */ +@Parameterized.Parameter(2) +public boolean withNearCache; + +/** */ +@Parameterized.Parameters(name = "nodes={0}, backups={1}, withNearCache={2}") +public static List params() { +List> nodesAndBackups = F.asList( Review Comment: `int[][]` is enough here. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071072854 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotTest.java: ## @@ -219,6 +221,8 @@ public void testFailIfSegmentNotFound() throws Exception { FileWriteAheadLogManager wal = (FileWriteAheadLogManager)srv.context().cache().context().wal(); +forceRollWal(srv); Review Comment: Why this change? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071071348 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutNoBackupMessagesBlockingTest.java: ## @@ -0,0 +1,82 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Stream; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** */ +@RunWith(Parameterized.class) +public class ConsistentCutNoBackupMessagesBlockingTest extends AbstractConsistentCutMessagesBlockingTest { +/** */ +@Parameterized.Parameter +public BlkNodeType txNodeBlkType; + +/** */ +@Parameterized.Parameter(1) +public BlkCutType cutBlkType; + +/** */ +@Parameterized.Parameter(2) +public BlkNodeType cutNodeBlkType; + +/** */ +@Parameterized.Parameters(name = "txNodeBlk={0}, cutBlkAt={1}, cutNodeBlk={2}") +public static List params() { +List p = new ArrayList<>(); + +Stream.of(BlkNodeType.NEAR, BlkNodeType.PRIMARY).forEach(txN -> +Stream.of(BlkNodeType.NEAR, BlkNodeType.PRIMARY).forEach(cutN -> { +for (BlkCutType c : BlkCutType.values()) +p.add(new Object[] {txN, c, cutN}); +}) +); + +return p; +} + +/** */ +@Test +public void testExplicitTransactions() throws Exception { Review Comment: Why this test case specific to this class only? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071014604 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/TransactionTestCase.java: ## @@ -0,0 +1,167 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiTuple; +import org.jetbrains.annotations.Nullable; + +/** + * Class represents a transaction test case for single or two keys. + */ +public class TransactionTestCase { +/** Keys array described with pairs (primary, backup). */ +@GridToStringInclude +private final T2[] keys; + +/** */ +private TransactionTestCase(T2[] keys) { +this.keys = keys; +} + +/** Test case with single key. */ +private static TransactionTestCase forSignleKey(Integer primary, @Nullable Integer backup) { +return new TransactionTestCase(new T2[] {new T2<>(primary, backup)}); +} + +/** Test case with two keys. */ +private static TransactionTestCase forTwoKeys(TransactionTestCase firstKey, TransactionTestCase secondKey) { +return new TransactionTestCase(new T2[] {firstKey.keys[0], secondKey.keys[0]}); +} + +/** + * Builds test cases for checking Consistency Cut. + * + * @param nodesCnt Count of nodes that participated in a test case. + * @param withBakup If {@code false} then no backups. + * @return List of test cases. + */ +public static List buildTestCases(int nodesCnt, boolean withBakup) { +List cases = new ArrayList<>(); + +// One entry. +int backups = withBakup ? nodesCnt : 0; + +for (int primary = 0; primary < nodesCnt; primary++) { +for (int backup = 0; backup <= backups; backup++) { +if (withBakup && (backup == primary || backup == nodesCnt)) +continue; + +cases.add(forSignleKey(primary, withBakup ? backup : null)); +} +} + +// Two entries. +int casesSize = cases.size(); + +for (int p1 = 0; p1 < casesSize; p1++) { +for (int p2 = 0; p2 < casesSize; p2++) +cases.add(forTwoKeys(cases.get(p1), cases.get(p2))); +} + +return cases; +} + +/** */ +public int firstKeyPrimary() { +return keys[0].getKey(); +} + +/** */ +public int firstKeyBackup() { +return keys[0].getValue(); +} + +/** */ +public int[] keys(IgniteEx grid, String cache) { +int[] k = new int[keys.length]; + +for (int i = 0; i < keys.length; i++) +k[i] = key(grid, cache, i); + +return k; +} + +/** + * Provides a key that for an existing partitioning schema match specified primary and backup node. + * + * @param grid Ignite grid. + * @param cache Cache name. + * @param keyNum Number of key in the test case. Review Comment: keyNum -> idx? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071014350 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/TransactionTestCase.java: ## @@ -0,0 +1,167 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiTuple; +import org.jetbrains.annotations.Nullable; + +/** + * Class represents a transaction test case for single or two keys. + */ +public class TransactionTestCase { +/** Keys array described with pairs (primary, backup). */ +@GridToStringInclude +private final T2[] keys; + +/** */ +private TransactionTestCase(T2[] keys) { +this.keys = keys; +} + +/** Test case with single key. */ +private static TransactionTestCase forSignleKey(Integer primary, @Nullable Integer backup) { +return new TransactionTestCase(new T2[] {new T2<>(primary, backup)}); +} + +/** Test case with two keys. */ +private static TransactionTestCase forTwoKeys(TransactionTestCase firstKey, TransactionTestCase secondKey) { +return new TransactionTestCase(new T2[] {firstKey.keys[0], secondKey.keys[0]}); +} + +/** + * Builds test cases for checking Consistency Cut. + * + * @param nodesCnt Count of nodes that participated in a test case. + * @param withBakup If {@code false} then no backups. + * @return List of test cases. + */ +public static List buildTestCases(int nodesCnt, boolean withBakup) { +List cases = new ArrayList<>(); + +// One entry. +int backups = withBakup ? nodesCnt : 0; + +for (int primary = 0; primary < nodesCnt; primary++) { +for (int backup = 0; backup <= backups; backup++) { +if (withBakup && (backup == primary || backup == nodesCnt)) +continue; + +cases.add(forSignleKey(primary, withBakup ? backup : null)); +} +} + +// Two entries. +int casesSize = cases.size(); + +for (int p1 = 0; p1 < casesSize; p1++) { +for (int p2 = 0; p2 < casesSize; p2++) +cases.add(forTwoKeys(cases.get(p1), cases.get(p2))); +} + +return cases; +} + +/** */ +public int firstKeyPrimary() { +return keys[0].getKey(); +} + +/** */ +public int firstKeyBackup() { +return keys[0].getValue(); +} + +/** */ +public int[] keys(IgniteEx grid, String cache) { +int[] k = new int[keys.length]; + +for (int i = 0; i < keys.length; i++) +k[i] = key(grid, cache, i); + +return k; +} + +/** + * Provides a key that for an existing partitioning schema match specified primary and backup node. + * + * @param grid Ignite grid. + * @param cache Cache name. + * @param keyNum Number of key in the test case. + * @return Key that matches specified primary and backup nodes. + */ +private int key(IgniteEx grid, String cache, int keyNum) { Review Comment: Single usage, can be inlined. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071012059 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/TransactionTestCase.java: ## @@ -0,0 +1,167 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiTuple; +import org.jetbrains.annotations.Nullable; + +/** + * Class represents a transaction test case for single or two keys. + */ +public class TransactionTestCase { +/** Keys array described with pairs (primary, backup). */ Review Comment: What is "primary" and "backup" means in your tests? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071009735 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/TransactionTestCase.java: ## @@ -0,0 +1,167 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiTuple; +import org.jetbrains.annotations.Nullable; + +/** + * Class represents a transaction test case for single or two keys. + */ +public class TransactionTestCase { +/** Keys array described with pairs (primary, backup). */ +@GridToStringInclude +private final T2[] keys; Review Comment: It seems like `Integer[][]` is enough here. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071008197 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/AbstractConsistentCutMessagesBlockingTest.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.internal.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.List; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; +import org.apache.ignite.transactions.TransactionConcurrency; + +import static org.apache.ignite.internal.processors.cache.consistentcut.AbstractConsistentCutBlockingTest.BlkNodeType.BACKUP; +import static org.apache.ignite.internal.processors.cache.consistentcut.AbstractConsistentCutBlockingTest.BlkNodeType.NEAR; +import static org.apache.ignite.internal.processors.cache.consistentcut.AbstractConsistentCutBlockingTest.BlkNodeType.PRIMARY; + +/** */ +public abstract class AbstractConsistentCutMessagesBlockingTest extends AbstractConsistentCutBlockingTest { +/** */ +private static Class txMsgBlkCls; + +/** {@inheritDoc} */ +@Override protected IgniteConfiguration getConfiguration(String instanceName) throws Exception { +IgniteConfiguration cfg = super.getConfiguration(instanceName); + +cfg.setCommunicationSpi(new TestRecordingCommunicationSpi()); + +return cfg; +} + +/** Initialize latches for test cases with blocking tx messages. */ +protected final void initMsgCase(Class msgCls, BlkNodeType txBlkNode, BlkCutType cutBlkType, BlkNodeType cutBlkNode) { +txBlkNodeType = txBlkNode; + +txMsgBlkCls = msgCls; +AbstractConsistentCutBlockingTest.cutBlkType = cutBlkType; +cutBlkNodeType = cutBlkNode; +} + +/** */ +@Override protected void runCase(TransactionTestCase testCase, int nearNode, TransactionConcurrency txConcurrency) throws Exception { +int txBlkNodeId = blkNode(nearNode, txBlkNodeType, testCase); + +int cutBlkNodeId = -1; + +if (cutBlkType != BlkCutType.NONE) +cutBlkNodeId = blkNode(nearNode, cutBlkNodeType, testCase); + +if (skipMsgTestCase(txBlkNodeId, nearNode, testCase)) +return; + +log.info("START CASE " + caseNum + +". Data=" + testCase + +", nearNode=" + nearNode + +", txBlkNodeId=" + txBlkNodeId + +", txBlkNodeType=" + txBlkNodeType + +", cutBlkNodeId=" + cutBlkNodeId + +", msg=" + txMsgBlkCls.getSimpleName()); + +run(() -> tx(nearNode, testCase, txConcurrency), txBlkNodeId, cutBlkNodeId); +} + +/** {@inheritDoc} */ +@Override protected void blockTx(IgniteEx blkNode) { +TestRecordingCommunicationSpi.spi(blkNode).blockMessages((n, msg) -> +msg.getClass().equals(txMsgBlkCls) +); +} + +/** {@inheritDoc} */ +@Override protected void unblockTx(IgniteEx blkNode) { +TestRecordingCommunicationSpi.spi(blkNode).stopBlock(); +} + +/** {@inheritDoc} */ +@Override protected void awaitTxBlocked(IgniteEx blkNode) throws Exception { +TestRecordingCommunicationSpi.spi(blkNode).waitForBlocked(1); +} + +/** */ +private boolean skipMsgTestCase(int txBlkNodeId, int nearNodeId, TransactionTestCase testCase) { Review Comment: Please, describe the e
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071005323 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/TransactionTestCase.java: ## @@ -0,0 +1,167 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiTuple; +import org.jetbrains.annotations.Nullable; + +/** + * Class represents a transaction test case for single or two keys. + */ +public class TransactionTestCase { +/** Keys array described with pairs (primary, backup). */ +@GridToStringInclude +private final T2[] keys; + +/** */ +private TransactionTestCase(T2[] keys) { +this.keys = keys; +} + +/** Test case with single key. */ +private static TransactionTestCase forSignleKey(Integer primary, @Nullable Integer backup) { +return new TransactionTestCase(new T2[] {new T2<>(primary, backup)}); +} + +/** Test case with two keys. */ +private static TransactionTestCase forTwoKeys(TransactionTestCase firstKey, TransactionTestCase secondKey) { Review Comment: Single usage, can be inlined -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1071005076 ## modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistentcut/TransactionTestCase.java: ## @@ -0,0 +1,167 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiTuple; +import org.jetbrains.annotations.Nullable; + +/** + * Class represents a transaction test case for single or two keys. + */ +public class TransactionTestCase { +/** Keys array described with pairs (primary, backup). */ +@GridToStringInclude +private final T2[] keys; + +/** */ +private TransactionTestCase(T2[] keys) { +this.keys = keys; +} + +/** Test case with single key. */ +private static TransactionTestCase forSignleKey(Integer primary, @Nullable Integer backup) { Review Comment: Single usage, can be inlined -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1069516677 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java: ## @@ -3130,6 +3143,50 @@ public DistributedTransactionConfiguration getDistributedTransactionConfiguratio return distributedTransactionConfiguration; } +/** @param transform Transaction message transformer. */ +public void txMessageTransformer(BiFunction transform) { +txMsgTransform = transform; +} + +/** @param callback Callback invoked on transaction commit. */ +public void onCommitCallback(Consumer callback) { +onCommitCallback = callback; +} + +/** + * Sends transaction message after transforming it. + * + * @param nodeId Node ID to send message. + * @param msg Original message to transform. + * @param tx Transaction. + * @param plc IO policy. + */ +public void sendTransactionMessage(UUID nodeId, GridCacheMessage msg, IgniteInternalTx tx, byte plc) throws IgniteCheckedException { +BiFunction transform = txMsgTransform; + +if (transform != null) +msg = transform.apply(msg, tx); + +cctx.io().send(nodeId, msg, plc); +} + +/** + * Sends transaction message after transforming it. + * + * @param n Node to send message. + * @param msg Original message to transform. + * @param tx Transaction. + * @param plc IO policy. + */ +public void sendTransactionMessage(ClusterNode n, GridCacheMessage msg, IgniteInternalTx tx, byte plc) throws IgniteCheckedException { +BiFunction transform = txMsgTransform; Review Comment: Can we call `sendTransactionMessage(n.id(), msg, tx, plc)` instead of having double implementation? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1069500134 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java: ## @@ -879,6 +898,33 @@ private IgniteInternalFuture initLocalSnapshotStartSt return initLocalFullSnapshot(req, grpIds, comprGrpIds, withMetaStorage); } +/** + * Handles received Consistent Cut ID from remote node. + * + * @param id Consistent Cut ID. + */ +public void handleConsistentCutId(UUID id) { +if (consistentCutId != null) +return; + +synchronized (snpOpMux) { +if (consistentCutId != null) +return; Review Comment: Let's check if `id.equals(consitentCutId)` and print some warning if not. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1069491997 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java: ## @@ -3130,6 +3143,50 @@ public DistributedTransactionConfiguration getDistributedTransactionConfiguratio return distributedTransactionConfiguration; } +/** @param transform Transaction message transformer. */ +public void txMessageTransformer(BiFunction transform) { +txMsgTransform = transform; +} + +/** @param callback Callback invoked on transaction commit. */ +public void onCommitCallback(Consumer callback) { +onCommitCallback = callback; +} + +/** + * Sends transaction message after transforming it. + * + * @param nodeId Node ID to send message. + * @param msg Original message to transform. + * @param tx Transaction. + * @param plc IO policy. + */ +public void sendTransactionMessage(UUID nodeId, GridCacheMessage msg, IgniteInternalTx tx, byte plc) throws IgniteCheckedException { Review Comment: `plc` paramter always equal to `tx.ioPolicy()` and can be eliminated. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1069352124 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java: ## @@ -218,73 +222,151 @@ public IgniteTxHandler(GridCacheSharedContext ctx) { txPrepareMsgLog = ctx.logger(CU.TX_MSG_PREPARE_LOG_CATEGORY); txFinishMsgLog = ctx.logger(CU.TX_MSG_FINISH_LOG_CATEGORY); -ctx.io().addCacheHandler(0, GridNearTxPrepareRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxPrepareRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxPrepareRequest(nodeId, (GridNearTxPrepareRequest)msg); } }); -ctx.io().addCacheHandler(0, GridNearTxPrepareResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxPrepareResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxPrepareResponse(nodeId, (GridNearTxPrepareResponse)msg); } }); -ctx.io().addCacheHandler(0, GridNearTxFinishRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxFinishRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxFinishRequest(nodeId, (GridNearTxFinishRequest)msg); } }); -ctx.io().addCacheHandler(0, GridNearTxFinishResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxFinishResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxFinishResponse(nodeId, (GridNearTxFinishResponse)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxPrepareRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxPrepareRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxPrepareRequest(nodeId, (GridDhtTxPrepareRequest)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxPrepareResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxPrepareResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxPrepareResponse(nodeId, (GridDhtTxPrepareResponse)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxFinishRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxFinishRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxFinishRequest(nodeId, (GridDhtTxFinishRequest)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxOnePhaseCommitAckRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxOnePhaseCommitAckRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxOnePhaseCommitAckRequest(nodeId, (GridDhtTxOnePhaseCommitAckRequest)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxFinishResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxFinishResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxFinishResponse(nodeId, (GridDhtTxFinishResponse)msg); } }); -ctx.io().addCacheHandler(0, GridCacheTxRecoveryRequest.class, +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridCacheTxRecoveryRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheTxRecoveryRequest req) { processCheckPreparedTxRequest(nodeId, req); } }); -ctx.io().addCacheHandler(0, GridCacheTxRecoveryResponse.class, +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridCacheTxRecoveryResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheTxRecoveryResponse res) { processCheckPreparedTxResponse(nodeId, res); } }); + +ctx.io().addCacheHandler(TX_MSG_HND_ID, ConsistentCutAwareMessage.class, +new CI2() { +@Override public void apply(UUID nodeId, ConsistentCutAwareMessage msg) { +processConsistentCutAwareMessage(nodeId, msg); +} +}); +} + +/** */ +private void processConsistentCutAwareMessage(UUID nodeId, ConsistentCutAwareMessage msg) { +
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1069255142 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java: ## @@ -1293,6 +1303,25 @@ WALRecord readPlainRecord(RecordType type, ByteBufferBackedDataInput in, break; +case CONSISTENT_CUT_START_RECORD: +long mst = in.readLong(); +long lst = in.readLong(); + +res = new ConsistentCutStartRecord(new UUID(mst, lst)); + +break; + +case CONSISTENT_CUT_FINISH_RECORD: +long mstSignBits = in.readLong(); +long lstSignBits = in.readLong(); + +Set before = readVersions(in); Review Comment: before -> included ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java: ## @@ -1293,6 +1303,25 @@ WALRecord readPlainRecord(RecordType type, ByteBufferBackedDataInput in, break; +case CONSISTENT_CUT_START_RECORD: +long mst = in.readLong(); +long lst = in.readLong(); + +res = new ConsistentCutStartRecord(new UUID(mst, lst)); + +break; + +case CONSISTENT_CUT_FINISH_RECORD: +long mstSignBits = in.readLong(); +long lstSignBits = in.readLong(); + +Set before = readVersions(in); +Set after = readVersions(in); Review Comment: after -> excluded -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1069251409 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,106 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * This record is written to WAL after Consistent Cut finished on a baseline node. + * + * It guarantees that the BEFORE side consists of: Review Comment: During recovery node must recovery: 1. Transactions committed before {@link ConsistentCutStartRecord} except those contained in {@link #excluded()}. 2. Transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} and contained in {@link #included()}. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1066132630 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java: ## @@ -509,7 +509,7 @@ private boolean finish(boolean commit, + n.id())); } else { -cctx.io().send(n, req, tx.ioPolicy()); +cctx.io().send(n, cctx.snapshotMgr().wrapMessage(req, tx.cutId()), tx.ioPolicy()); Review Comment: Can we implement some kind of plug in message wrapper inside of `GridCacheIoManager` ? ``` private volatile Function msgProc; public void send(ClusterNode node, GridCacheMessage msg, byte plc) throws IgniteCheckedException { assert !node.isLocal() : node; Function msgProc0 = msgProc if (msgProc0 != null) msg = msgProc0.apply(msg); ``` It seems you can access current transaction with the ThreadLocal - `IgniteTxManager#threadCtx`. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1066088576 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java: ## @@ -218,73 +222,151 @@ public IgniteTxHandler(GridCacheSharedContext ctx) { txPrepareMsgLog = ctx.logger(CU.TX_MSG_PREPARE_LOG_CATEGORY); txFinishMsgLog = ctx.logger(CU.TX_MSG_FINISH_LOG_CATEGORY); -ctx.io().addCacheHandler(0, GridNearTxPrepareRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxPrepareRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxPrepareRequest(nodeId, (GridNearTxPrepareRequest)msg); } }); -ctx.io().addCacheHandler(0, GridNearTxPrepareResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxPrepareResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxPrepareResponse(nodeId, (GridNearTxPrepareResponse)msg); } }); -ctx.io().addCacheHandler(0, GridNearTxFinishRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxFinishRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxFinishRequest(nodeId, (GridNearTxFinishRequest)msg); } }); -ctx.io().addCacheHandler(0, GridNearTxFinishResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxFinishResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxFinishResponse(nodeId, (GridNearTxFinishResponse)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxPrepareRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxPrepareRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxPrepareRequest(nodeId, (GridDhtTxPrepareRequest)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxPrepareResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxPrepareResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxPrepareResponse(nodeId, (GridDhtTxPrepareResponse)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxFinishRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxFinishRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxFinishRequest(nodeId, (GridDhtTxFinishRequest)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxOnePhaseCommitAckRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxOnePhaseCommitAckRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxOnePhaseCommitAckRequest(nodeId, (GridDhtTxOnePhaseCommitAckRequest)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxFinishResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxFinishResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxFinishResponse(nodeId, (GridDhtTxFinishResponse)msg); } }); -ctx.io().addCacheHandler(0, GridCacheTxRecoveryRequest.class, +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridCacheTxRecoveryRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheTxRecoveryRequest req) { processCheckPreparedTxRequest(nodeId, req); } }); -ctx.io().addCacheHandler(0, GridCacheTxRecoveryResponse.class, +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridCacheTxRecoveryResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheTxRecoveryResponse res) { processCheckPreparedTxResponse(nodeId, res); } }); + +ctx.io().addCacheHandler(TX_MSG_HND_ID, ConsistentCutAwareMessage.class, +new CI2() { +@Override public void apply(UUID nodeId, ConsistentCutAwareMessage msg) { +processConsistentCutAwareMessage(nodeId, msg); +} +}); +} + +/** */ +private void processConsistentCutAwareMessage(UUID nodeId, ConsistentCutAwareMessage msg) { +
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1066085859 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java: ## @@ -1608,6 +1608,9 @@ public void commitTx(IgniteInternalTx tx) throws IgniteCheckedException { ", tx=" + tx.getClass().getSimpleName() + ']'); } +if (!cctx.kernalContext().clientNode()) Review Comment: Can we have something like following instead of invoke `onCommit` for every commit. ``` private volatile Consumer commitCb; ... public void commitTx(IgniteInternalTx tx) throws IgniteCheckedException { Consumer commitCb0 = commitCb; if (commitCb0 != null) commitCb0.accept(tx); } ``` So `SnapshotManager` can set and clear callback when required. And all other time we enlarge tx commit to one volatile null check. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1066076820 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/ConsistentCutMarkWalFuture.java: ## @@ -0,0 +1,193 @@ +/* + * 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.processors.cache.persistence.snapshot; + +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; + +/** Writes Consistent Cut WAL records. Future completes with pointer to {@link ConsistentCutFinishRecord}. */ +class ConsistentCutMarkWalFuture extends GridFutureAdapter { +/** Cache context. */ +private final GridCacheSharedContext cctx; + +/** Consistent Cut ID. */ +private final UUID id; + +/** Logger. */ +private final IgniteLogger log; + +/** Set of checked transactions belong to the BEFORE set. */ +@GridToStringInclude +private Set before; + +/** Set of checked transactions belong to the AFTER set. */ +@GridToStringInclude +private Set after; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedFromActive = ConcurrentHashMap.newKeySet(); + +/** */ +ConsistentCutMarkWalFuture(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(ConsistentCutMarkWalFuture.class); +} + +/** Inits the future: it prepares list of active transactions to check which side of Consistent Cut they belong to. */ +protected void init() { +try { +cctx.wal().log(new ConsistentCutStartRecord(id)); + +before = ConcurrentHashMap.newKeySet(); +after = ConcurrentHashMap.newKeySet(); + +GridCompoundFuture checkFut = new GridCompoundFuture<>(CU.boolReducer()); + +Iterator> finFutIt = cctx.tm().activeTransactions().stream() +.filter(tx -> tx.state() != ACTIVE) +.map(IgniteInternalTx::finishFuture) +.iterator(); + +// Invoke sequentially over two iterators: +// 1. Iterators are weakly consistent. +// 2. We need a guarantee to handle `removedFromActive` after `activeTxs` to avoid missed transactions. +checkTransactions(finFutIt, checkFut); + +Iterator> removedFromActiveIter = removedFromActive.iterator(); +removedFromActive = null; Review Comment: Empty line before and after required -- This is an automated message from the Apache Git Service. To res
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1066068742 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/ConsistentCutMarkWalFuture.java: ## @@ -0,0 +1,193 @@ +/* + * 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.processors.cache.persistence.snapshot; + +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; + +/** Writes Consistent Cut WAL records. Future completes with pointer to {@link ConsistentCutFinishRecord}. */ +class ConsistentCutMarkWalFuture extends GridFutureAdapter { +/** Cache context. */ +private final GridCacheSharedContext cctx; + +/** Consistent Cut ID. */ +private final UUID id; + +/** Logger. */ +private final IgniteLogger log; + +/** Set of checked transactions belong to the BEFORE set. */ +@GridToStringInclude +private Set before; + +/** Set of checked transactions belong to the AFTER set. */ +@GridToStringInclude +private Set after; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedFromActive = ConcurrentHashMap.newKeySet(); + +/** */ +ConsistentCutMarkWalFuture(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(ConsistentCutMarkWalFuture.class); +} + +/** Inits the future: it prepares list of active transactions to check which side of Consistent Cut they belong to. */ +protected void init() { +try { +cctx.wal().log(new ConsistentCutStartRecord(id)); + +before = ConcurrentHashMap.newKeySet(); +after = ConcurrentHashMap.newKeySet(); + +GridCompoundFuture checkFut = new GridCompoundFuture<>(CU.boolReducer()); + +Iterator> finFutIt = cctx.tm().activeTransactions().stream() +.filter(tx -> tx.state() != ACTIVE) +.map(IgniteInternalTx::finishFuture) +.iterator(); + +// Invoke sequentially over two iterators: +// 1. Iterators are weakly consistent. +// 2. We need a guarantee to handle `removedFromActive` after `activeTxs` to avoid missed transactions. +checkTransactions(finFutIt, checkFut); + +Iterator> removedFromActiveIter = removedFromActive.iterator(); +removedFromActive = null; +checkTransactions(removedFromActiveIter, checkFut); + +checkFut.markInitialized(); + +check
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1066064232 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/ConsistentCutMarkWalFuture.java: ## @@ -0,0 +1,193 @@ +/* + * 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.processors.cache.persistence.snapshot; + +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; + +/** Writes Consistent Cut WAL records. Future completes with pointer to {@link ConsistentCutFinishRecord}. */ +class ConsistentCutMarkWalFuture extends GridFutureAdapter { +/** Cache context. */ +private final GridCacheSharedContext cctx; + +/** Consistent Cut ID. */ +private final UUID id; + +/** Logger. */ +private final IgniteLogger log; + +/** Set of checked transactions belong to the BEFORE set. */ +@GridToStringInclude +private Set before; + +/** Set of checked transactions belong to the AFTER set. */ +@GridToStringInclude +private Set after; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedFromActive = ConcurrentHashMap.newKeySet(); + +/** */ +ConsistentCutMarkWalFuture(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(ConsistentCutMarkWalFuture.class); +} + +/** Inits the future: it prepares list of active transactions to check which side of Consistent Cut they belong to. */ +protected void init() { +try { +cctx.wal().log(new ConsistentCutStartRecord(id)); + +before = ConcurrentHashMap.newKeySet(); Review Comment: Please, move initialization to declaration. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1066060597 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/ConsistentCut.java: ## @@ -0,0 +1,130 @@ +/* + * 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.processors.cache.persistence.snapshot; + +import java.util.UUID; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Consistent Cut is a distributed algorithm that defines two set of transactions - BEFORE and AFTER cut - on baseline nodes. + * It guarantees that every transaction was included into BEFORE on one node also included into the BEFORE on every other node + * participated in the transaction. It means that Ignite nodes can safely recover themselves to the consistent BEFORE + * state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages wrapped in {@link ConsistentCutAwareMessage}. + * + * There are two roles that node can play: + * 1. Wraps outgoing transaction messages into {@link ConsistentCutAwareMessage}. For this role every node is responsible. + * 2. Writes Consistent Cut records into WAL. Only baseline nodes do it. + * + * Nodes start wrapping messages from the moment Consistent Cut started on the node, and finsihed after all baseline + * nodes complete their role 2. + */ +public class ConsistentCut { +/** Grid cache context. */ +private final GridCacheSharedContext cctx; + +/** Consistent Cut ID. */ +private final UUID id; + +/** + * If {@code true} it wraps messages into {@link ConsistentCutAwareMessage}. Becames {@code false} after every baseline node + * completes {@link #markWalFut}. + */ +private volatile boolean wrapMsg = true; + +/** Future that completes after last {@link ConsistentCutAwareMessage} was sent. */ +private final GridFutureAdapter wrapMsgsFut = new GridFutureAdapter<>(); + +/** Future that completes after {@link ConsistentCutFinishRecord} was written. */ +private final @Nullable ConsistentCutMarkWalFuture markWalFut; + +/** */ +ConsistentCut(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +markWalFut = baselineNode(cctx.localNode(), cctx.kernalContext().state().clusterState()) +? new ConsistentCutMarkWalFuture(cctx, id) : null; +} + +/** + * Wraps a transaction message if needed. + * + * @param txMsg Transaction message to wrap. + * @param txCutId Consistent Cut ID after which transaction committed, if specified. + */ +GridCacheMessage wrapMessage(GridCacheMessage txMsg, @Nullable UUID txCutId) { +if (wrapMsg) +return new ConsistentCutAwareMessage(txMsg, id, txCutId); + +return txMsg; +} + +/** Stops wrapping outging messages. */ +void stopWrapMessages() { +wrapMsg = false; + +GridCompoundIdentityFuture activeTxsFut = new GridCompoundIdentityFuture<>(); + +for (IgniteInternalTx tx: cctx.tm().activeTransactions()) +activeTxsFut.add(tx.finishFuture()); + +activeTxsFut.markInitialized(); + +activeTxsFut.listen(f -> wrapMsgsFut.onDone()); +} + +/** */ +void init(GridCacheSharedContext cctx) { Review Comment: Parameter useless. `cctx` instance cached in c
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1066048347 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/ConsistentCut.java: ## @@ -0,0 +1,130 @@ +/* + * 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.processors.cache.persistence.snapshot; + +import java.util.UUID; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Consistent Cut is a distributed algorithm that defines two set of transactions - BEFORE and AFTER cut - on baseline nodes. + * It guarantees that every transaction was included into BEFORE on one node also included into the BEFORE on every other node + * participated in the transaction. It means that Ignite nodes can safely recover themselves to the consistent BEFORE + * state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages wrapped in {@link ConsistentCutAwareMessage}. + * + * There are two roles that node can play: + * 1. Wraps outgoing transaction messages into {@link ConsistentCutAwareMessage}. For this role every node is responsible. + * 2. Writes Consistent Cut records into WAL. Only baseline nodes do it. + * + * Nodes start wrapping messages from the moment Consistent Cut started on the node, and finsihed after all baseline + * nodes complete their role 2. + */ +public class ConsistentCut { Review Comment: With current implementation, purpose of this class is not clear. It seems we can simply move all the code to `IgniteSnapshotManager`. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1066030836 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,106 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * This record is written to WAL after Consistent Cut finished on a baseline node. + * + * It guarantees that the BEFORE side consists of: + * 1. Transactions committed before {@link ConsistentCutStartRecord} and weren't included into {@link #after()}. + * 2. Transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and were included into {@link #before()}. + * + * It guarantees that the AFTER side consists of: + * 1. Transactions committed before {@link ConsistentCutStartRecord} and were included into {@link #after()}. + * 2. Transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and weren't included into {@link #before()}. + */ +public class ConsistentCutFinishRecord extends WALRecord { +/** Consistent Cut ID. */ +@GridToStringInclude +private final UUID cutId; Review Comment: The same field inside `ConsistentCut` class names `id`. Let's rename this field to `id` also. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1065978497 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,106 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * This record is written to WAL after Consistent Cut finished on a baseline node. + * + * It guarantees that the BEFORE side consists of: + * 1. Transactions committed before {@link ConsistentCutStartRecord} and weren't included into {@link #after()}. + * 2. Transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and were included into {@link #before()}. + * + * It guarantees that the AFTER side consists of: + * 1. Transactions committed before {@link ConsistentCutStartRecord} and were included into {@link #after()}. + * 2. Transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and weren't included into {@link #before()}. + */ +public class ConsistentCutFinishRecord extends WALRecord { +/** Consistent Cut ID. */ +@GridToStringInclude +private final UUID cutId; + +/** + * Set of transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + * to include to the BEFORE side of Consistent Cut. + */ +@GridToStringInclude +private final Set before; + +/** + * Set of transactions committed before {@link ConsistentCutStartRecord} to include to the AFTER side of Consistent Cut. + */ +@GridToStringInclude +private final Set after; Review Comment: Let's rename to `excluded` or `excludedTxs` -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1065978206 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,106 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * This record is written to WAL after Consistent Cut finished on a baseline node. + * + * It guarantees that the BEFORE side consists of: + * 1. Transactions committed before {@link ConsistentCutStartRecord} and weren't included into {@link #after()}. + * 2. Transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and were included into {@link #before()}. + * + * It guarantees that the AFTER side consists of: + * 1. Transactions committed before {@link ConsistentCutStartRecord} and were included into {@link #after()}. + * 2. Transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and weren't included into {@link #before()}. + */ +public class ConsistentCutFinishRecord extends WALRecord { +/** Consistent Cut ID. */ +@GridToStringInclude +private final UUID cutId; + +/** + * Set of transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + * to include to the BEFORE side of Consistent Cut. + */ +@GridToStringInclude +private final Set before; Review Comment: Let's rename to `included` or `includedTxs` -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1065977763 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,106 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * This record is written to WAL after Consistent Cut finished on a baseline node. + * + * It guarantees that the BEFORE side consists of: + * 1. Transactions committed before {@link ConsistentCutStartRecord} and weren't included into {@link #after()}. + * 2. Transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and were included into {@link #before()}. + * + * It guarantees that the AFTER side consists of: + * 1. Transactions committed before {@link ConsistentCutStartRecord} and were included into {@link #after()}. + * 2. Transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and weren't included into {@link #before()}. + */ +public class ConsistentCutFinishRecord extends WALRecord { +/** Consistent Cut ID. */ +@GridToStringInclude +private final UUID cutId; + +/** + * Set of transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + * to include to the BEFORE side of Consistent Cut. + */ +@GridToStringInclude +private final Set before; + +/** + * Set of transactions committed before {@link ConsistentCutStartRecord} to include to the AFTER side of Consistent Cut. + */ +@GridToStringInclude +private final Set after; + +/** */ +public ConsistentCutFinishRecord(UUID cutId, Set before, Set after) { +this.cutId = cutId; +this.before = before; +this.after = after; +} + +/** */ +public Set before() { +return before; +} + +/** */ +public Set after() { +return after; +} + +/** */ +public UUID cutId() { +return cutId; +} + +/** {@inheritDoc} */ +@Override public RecordType type() { +return RecordType.CONSISTENT_CUT_FINISH_RECORD; +} + +/** + * Calculating the size of the record. + * + * @return Size in bytes. + */ +public int dataSize() { +int size = 8 + 8 + 4 + 4; // ID, before and after tx count. Review Comment: `8 + 8` is UUID size? Let's make it just 16. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1032226621 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/BaselineConsistentCut.java: ## @@ -0,0 +1,235 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +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 static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes Consistent Cut running on baseline nodes. + */ +public class BaselineConsistentCut implements ConsistentCut { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** Consistent Cut ID. */ +@GridToStringInclude +private final UUID id; + +/** Set of checked transactions belong to the BEFORE set. */ +@GridToStringInclude +private Set before; + +/** Set of checked transactions belong to the AFTER set. */ +@GridToStringInclude +private Set after; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedFromActive = ConcurrentHashMap.newKeySet(); + +/** Consistent Cut future, completes with pointer to {@link ConsistentCutFinishRecord}. */ +private final GridFutureAdapter fut = new GridFutureAdapter<>(); + +/** */ +BaselineConsistentCut(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(BaselineConsistentCut.class); + +fut.listen(r -> removedFromActive = null); +} + +/** + * Inits local Consistent Cut: prepares list of active transactions to check which side of Consistent Cut they belong to. + */ +protected void init() { +try { +cctx.wal().log(new ConsistentCutStartRecord(id)); + +before = ConcurrentHashMap.newKeySet(); +after = ConcurrentHashMap.newKeySet(); + +GridCompoundFuture checkFut = new GridCompoundFuture<>(CU.boolReducer()); + +Iterator> finFutIt = cctx.tm().activeTransactions().stream() +.filter(tx -> tx.state() != ACTIVE) +.map(IgniteInternalTx::finishFuture) +.iterator(); + +// Invoke sequentially over two iterators: +// 1. iterators are weakly consistent. Review Comment: Sentence must starts with capital letter. -- This is an automated message from the Apac
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1032227120 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/BaselineConsistentCut.java: ## @@ -0,0 +1,235 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +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 static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes Consistent Cut running on baseline nodes. + */ +public class BaselineConsistentCut implements ConsistentCut { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** Consistent Cut ID. */ +@GridToStringInclude +private final UUID id; + +/** Set of checked transactions belong to the BEFORE set. */ +@GridToStringInclude +private Set before; + +/** Set of checked transactions belong to the AFTER set. */ +@GridToStringInclude +private Set after; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedFromActive = ConcurrentHashMap.newKeySet(); + +/** Consistent Cut future, completes with pointer to {@link ConsistentCutFinishRecord}. */ +private final GridFutureAdapter fut = new GridFutureAdapter<>(); + +/** */ +BaselineConsistentCut(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(BaselineConsistentCut.class); + +fut.listen(r -> removedFromActive = null); +} + +/** + * Inits local Consistent Cut: prepares list of active transactions to check which side of Consistent Cut they belong to. + */ +protected void init() { +try { +cctx.wal().log(new ConsistentCutStartRecord(id)); + +before = ConcurrentHashMap.newKeySet(); +after = ConcurrentHashMap.newKeySet(); + +GridCompoundFuture checkFut = new GridCompoundFuture<>(CU.boolReducer()); + +Iterator> finFutIt = cctx.tm().activeTransactions().stream() +.filter(tx -> tx.state() != ACTIVE) +.map(IgniteInternalTx::finishFuture) +.iterator(); + +// Invoke sequentially over two iterators: +// 1. iterators are weakly consistent. +// 2. we need a guarantee to handle `removedFromActive` after `activeTxs` to avoid missed transac
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1030463126 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,194 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * Consistent Cut is a distributed algorithm that defines two set of transactions - BEFORE and AFTER cut - on baseline nodes. + * It guarantees that every transaction was included into BEFORE on one node also included into the BEFORE on every other node + * participated in the transaction. It means that Ignite nodes can safely recover themselves to the consistent BEFORE + * state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCut} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} after which it committed. The value is initially set on + *originated node for two-phase-commit, and backup node for one-phase-commit. Then it is propagated to other nodes with + *{@link ConsistentCutAwareMessage}. + * 4. On baseline nodes it awaits {@link BaselineConsistentCut}, that completes with pointer to {@link ConsistentCutFinishRecord}. + * 5. After Consistent Cut finished on all nodes, it clears {@link ConsistentCut} and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** Current Consistent Cut, {@code null} if not running. */ +@GridToStringInclude +private final AtomicReference consistentCutRef = new AtomicReference<>(); + +/** ID of the last finished Consistent Cut. Required to avoid re-run Consistent Cut with the same id. */ +@GridToStringInclude +protected volatile UUID lastFinishedCutId; + +/** {@inheritDoc} */ +@Override public void start0() throws IgniteCheckedException { +super.start0(); + +cctx.exchange().registerExchangeAwareComponent(this); +} + +/** {@inheritDoc} */ +@Override public void stop0(boolean cancel) { +cancelConsistentCut(new IgniteCheckedException("Ignite node is stopping.")); +} + +/** + * Stops Consistent Cut in case of baseline topology changed. + */ +@Override public void onInitBeforeTopologyLock(GridDhtPartitionsExchangeFuture fut) { +if (fut.changedBaseline() || fut.isBaselineNodeFailed()) +cancelConsistentCut(new IgniteCheckedException("Ignite topology changed, can't finish Consistent Cut.
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1030212268 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,224 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * Consistent Cut is a distributed algorithm that defines two set of transactions - BEFORE and AFTER cut. It guarantees that every + * transaction committed BEFORE also will be committed BEFORE on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCutFuture} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 4. On baseline nodes in {@link BaselineConsistentCutFuture}: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 5. After Consistent Cut finished globally, it clears {@link ConsistentCutFuture} variable and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** Current Consistent Cut, {@code null} if not running. */ +private final AtomicReference cutFutRef = new AtomicReference<>(); + +/** ID of the last finished {@link ConsistentCutFuture}. Required to avoid re-run {@link ConsistentCutFuture} with the same id. */ +protected volatile UUID lastFinishedCutId; + +/** {@inheritDoc} */ +@Override public void start0() throws IgniteCheckedException { +super.start0(); + +cctx.exchange().
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1030212268 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,224 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * Consistent Cut is a distributed algorithm that defines two set of transactions - BEFORE and AFTER cut. It guarantees that every + * transaction committed BEFORE also will be committed BEFORE on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCutFuture} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 4. On baseline nodes in {@link BaselineConsistentCutFuture}: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 5. After Consistent Cut finished globally, it clears {@link ConsistentCutFuture} variable and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** Current Consistent Cut, {@code null} if not running. */ +private final AtomicReference cutFutRef = new AtomicReference<>(); + +/** ID of the last finished {@link ConsistentCutFuture}. Required to avoid re-run {@link ConsistentCutFuture} with the same id. */ +protected volatile UUID lastFinishedCutId; + +/** {@inheritDoc} */ +@Override public void start0() throws IgniteCheckedException { +super.start0(); + +cctx.exchange().
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1030207293 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,224 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * Consistent Cut is a distributed algorithm that defines two set of transactions - BEFORE and AFTER cut. It guarantees that every + * transaction committed BEFORE also will be committed BEFORE on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCutFuture} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 4. On baseline nodes in {@link BaselineConsistentCutFuture}: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 5. After Consistent Cut finished globally, it clears {@link ConsistentCutFuture} variable and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** Current Consistent Cut, {@code null} if not running. */ +private final AtomicReference cutFutRef = new AtomicReference<>(); + +/** ID of the last finished {@link ConsistentCutFuture}. Required to avoid re-run {@link ConsistentCutFuture} with the same id. */ +protected volatile UUID lastFinishedCutId; + +/** {@inheritDoc} */ +@Override public void start0() throws IgniteCheckedException { +super.start0(); + +cctx.exchange().
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1030201469 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutFuture.java: ## @@ -0,0 +1,31 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.UUID; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; + +/** + * Future that completes after Consistent Cut finished and return pointer to {@link ConsistentCutFinishRecord}. + */ +public interface ConsistentCutFuture extends IgniteInternalFuture { +/** Consistent Cut ID. */ +public UUID id(); Review Comment: Can we store id outside of `ConsistentCutFuture` and get rid of ConsistentCutFuture interface and ClientConsistentCutFuture? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1030198025 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/BaselineConsistentCutFuture.java: ## @@ -0,0 +1,215 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +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.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes Consistent Cut running on baseline nodes. + */ +public class BaselineConsistentCutFuture extends GridFutureAdapter implements ConsistentCutFuture { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** ID of Consistent Cut. */ +private final UUID id; Review Comment: Why this field not include in toString? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1029483232 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java: ## @@ -3228,6 +3231,38 @@ private void writeStatistics(IgniteInternalTx tx, boolean commited) { commited); } +/** + * Finds whether this local transaction instance commits first: + * - For 2PC transactions the commit order is direct order (from originated to primary and backup nodes). + * - For 1PC transactions the commit order is reverse order (from backup to primary and near nodes). + * + * @return Whether this transaction instance is first in the commit order. + */ +private boolean firstCommit(IgniteInternalTx tx) { +if (log.isDebugEnabled()) +log.debug("Check transaction for first commit: " + tx); Review Comment: This debug is useless. Please, remove it. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1029478166 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java: ## @@ -218,73 +224,154 @@ public IgniteTxHandler(GridCacheSharedContext ctx) { txPrepareMsgLog = ctx.logger(CU.TX_MSG_PREPARE_LOG_CATEGORY); txFinishMsgLog = ctx.logger(CU.TX_MSG_FINISH_LOG_CATEGORY); -ctx.io().addCacheHandler(0, GridNearTxPrepareRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxPrepareRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxPrepareRequest(nodeId, (GridNearTxPrepareRequest)msg); } }); -ctx.io().addCacheHandler(0, GridNearTxPrepareResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxPrepareResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxPrepareResponse(nodeId, (GridNearTxPrepareResponse)msg); } }); -ctx.io().addCacheHandler(0, GridNearTxFinishRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxFinishRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxFinishRequest(nodeId, (GridNearTxFinishRequest)msg); } }); -ctx.io().addCacheHandler(0, GridNearTxFinishResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridNearTxFinishResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processNearTxFinishResponse(nodeId, (GridNearTxFinishResponse)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxPrepareRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxPrepareRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxPrepareRequest(nodeId, (GridDhtTxPrepareRequest)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxPrepareResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxPrepareResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxPrepareResponse(nodeId, (GridDhtTxPrepareResponse)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxFinishRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxFinishRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxFinishRequest(nodeId, (GridDhtTxFinishRequest)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxOnePhaseCommitAckRequest.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxOnePhaseCommitAckRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxOnePhaseCommitAckRequest(nodeId, (GridDhtTxOnePhaseCommitAckRequest)msg); } }); -ctx.io().addCacheHandler(0, GridDhtTxFinishResponse.class, new CI2() { +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridDhtTxFinishResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxFinishResponse(nodeId, (GridDhtTxFinishResponse)msg); } }); -ctx.io().addCacheHandler(0, GridCacheTxRecoveryRequest.class, +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridCacheTxRecoveryRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheTxRecoveryRequest req) { processCheckPreparedTxRequest(nodeId, req); } }); -ctx.io().addCacheHandler(0, GridCacheTxRecoveryResponse.class, +ctx.io().addCacheHandler(TX_MSG_HND_ID, GridCacheTxRecoveryResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheTxRecoveryResponse res) { processCheckPreparedTxResponse(nodeId, res); } }); + +ctx.io().addCacheHandler(TX_MSG_HND_ID, ConsistentCutAwareMessage.class, +new CI2() { +@Override public void apply(UUID nodeId, ConsistentCutAwareMessage msg) { +processConsistentCutAwareMessage(nodeId, msg); +} +}); +} + +/** */ +private void processConsistentCutAwareMessage(UUID nodeId, ConsistentCutAwareMessage msg) { +
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1029182992 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,217 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * Consistent Cut is a distributed algorithm that defines two set of transactions - BEFORE and AFTER cut. It guarantees that every + * transaction committed BEFORE also will be committed BEFORE on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCutFuture} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 4. On baseline nodes: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 5. After Consistent Cut finished globally, it clears {@link ConsistentCutFuture} variable and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** Current Consistent Cut, {@code null} if not running. */ +private final AtomicReference cutFutRef = new AtomicReference<>(); + +/** ID of the last finished {@link ConsistentCutFuture}. Required to avoid re-run {@link ConsistentCutFuture} with the same id. */ +protected volatile UUID lastFinishedCutId; + +/** {@inheritDoc} */ +@Override public void start0() throws IgniteCheckedException { +super.start0(); + +cctx.exchange().registerExchangeAwareComponent(this); +
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1029174490 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,217 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * Consistent Cut is a distributed algorithm that defines two set of transactions - BEFORE and AFTER cut. It guarantees that every + * transaction committed BEFORE also will be committed BEFORE on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCutFuture} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 4. On baseline nodes: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 5. After Consistent Cut finished globally, it clears {@link ConsistentCutFuture} variable and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** Current Consistent Cut, {@code null} if not running. */ +private final AtomicReference cutFutRef = new AtomicReference<>(); + +/** ID of the last finished {@link ConsistentCutFuture}. Required to avoid re-run {@link ConsistentCutFuture} with the same id. */ +protected volatile UUID lastFinishedCutId; + +/** {@inheritDoc} */ +@Override public void start0() throws IgniteCheckedException { +super.start0(); + +cctx.exchange().registerExchangeAwareComponent(this); +
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1028288816 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java: ## @@ -3228,6 +3231,41 @@ private void writeStatistics(IgniteInternalTx tx, boolean commited) { commited); } +/** + * Finds whether this local transaction instance commits first: + * - For 2PC transactions the commit order is direct order (from originated to primary and backup nodes). + * - For 1PC transactions the commit order is reverse order (from backup to primary and near nodes). + * + * @return Whether this transaction instance is first in the commit order. + */ +private boolean firstCommit(IgniteInternalTx tx) { +if (log.isDebugEnabled()) { +log.debug("`firstCommit` " + tx.nearXidVersion().asIgniteUuid() + " " + getClass().getSimpleName() Review Comment: Please, use Ignite log format. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1028288205 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java: ## @@ -133,8 +136,11 @@ * Isolated logic to process cache messages. */ public class IgniteTxHandler { +/** */ +private static final int TX_MSG_HND_ID = 0; + /** Logger. */ -private IgniteLogger log; +private final IgniteLogger log; Review Comment: Please, revert unnecessary changes. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1028286697 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/ConsistentCutStartRecordSerializer.java: ## @@ -0,0 +1,62 @@ +/* + * 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.processors.cache.persistence.wal.serializer; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.UUID; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput; + +/** */ +public class ConsistentCutStartRecordSerializer { Review Comment: Let's make record SerDe more standart and put code inside RecordDataV1Serializer itself. ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/ConsistentCutFinishRecordSerializer.java: ## @@ -0,0 +1,105 @@ +/* + * 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.processors.cache.persistence.wal.serializer; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; +import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; + +import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.putVersion; +import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.readVersion; + +/** */ +public class ConsistentCutFinishRecordSerializer { Review Comment: Let's make record SerDe more standart and put code inside RecordDataV1Serializer itself. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1028283077 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotMetadata.java: ## @@ -36,9 +36,9 @@ public class IncrementalSnapshotMetadata implements Serializable { /** Snapshot name. */ private final String snpName; - + /** Increment index. */ -private final int incIdx; +private final long incIdx; Review Comment: This change is out of scope of current PR. Please, revert it. ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java: ## @@ -80,7 +80,7 @@ public class SnapshotOperationRequest implements Serializable { private final boolean incremental; /** Index of incremental snapshot. */ -private final int incIdx; +private final long incIdx; Review Comment: This change is out of scope of current PR. Please, revert it. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1028281515 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTaskResult.java: ## @@ -17,6 +17,21 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; -/** */ +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; + +/** Result of active phase of incremental snapshot task. */ class IncrementalSnapshotFutureTaskResult { +/** Pointer to {@link ConsistentCutFinishRecord}. */ +private final WALPointer incSnpPtr; + +/** */ +IncrementalSnapshotFutureTaskResult(WALPointer incSnpPtr) { +this.incSnpPtr = incSnpPtr; +} + +/** */ +public WALPointer snapshotPointer() { Review Comment: Let's rename this to actual meaning. - `consistentCutFinishPointer`? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1028281104 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTaskResult.java: ## @@ -17,6 +17,21 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; -/** */ +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; + +/** Result of active phase of incremental snapshot task. */ class IncrementalSnapshotFutureTaskResult { +/** Pointer to {@link ConsistentCutFinishRecord}. */ +private final WALPointer incSnpPtr; Review Comment: Let's rename this to actual meaning. - `consCutFinishPtr`? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1028279743 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java: ## @@ -417,6 +417,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { return false; writer.incrementState(); + Review Comment: Please, revert this changes. ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java: ## @@ -417,6 +417,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { return false; writer.incrementState(); + Review Comment: Please, revert these changes. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027737850 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java: ## @@ -239,7 +241,21 @@ public void onResult(UUID nodeId, GridDhtTxFinishResponse res) { try { boolean nodeStopping = X.hasCause(err, NodeStoppingException.class); -this.tx.tmFinish(err == null, nodeStopping || cctx.kernalContext().failure().nodeStopping(), false); +if (cctx.consistentCutMgr() != null && this.tx.currentPrepareFuture() != null) { Review Comment: Or, at least, extract wrap logic to the utility method. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027705622 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java: ## @@ -239,7 +241,21 @@ public void onResult(UUID nodeId, GridDhtTxFinishResponse res) { try { boolean nodeStopping = X.hasCause(err, NodeStoppingException.class); -this.tx.tmFinish(err == null, nodeStopping || cctx.kernalContext().failure().nodeStopping(), false); +if (cctx.consistentCutMgr() != null && this.tx.currentPrepareFuture() != null) { Review Comment: Can we a. rework logic to avoid `cctx.consistentCutMgr() != null` checks. b. use NoOp or disabled implementation of cut manager in case it can't be used. ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java: ## @@ -239,7 +241,21 @@ public void onResult(UUID nodeId, GridDhtTxFinishResponse res) { try { boolean nodeStopping = X.hasCause(err, NodeStoppingException.class); -this.tx.tmFinish(err == null, nodeStopping || cctx.kernalContext().failure().nodeStopping(), false); +if (cctx.consistentCutMgr() != null && this.tx.currentPrepareFuture() != null) { Review Comment: Can we? a. rework logic to avoid `cctx.consistentCutMgr() != null` checks. b. use NoOp or disabled implementation of cut manager in case it can't be used. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027701749 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,212 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * ConsistentCut is a distributed algorithm that splits WAL on 2 global areas - BEFORE and AFTER. It guarantees that every + * transaction committed Before also will be committed Before on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCut} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 3. On baseline nodes: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 8. After Consistent Cut finished globally, it clears {@link ConsistentCut} variable and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** It serves updates of {@link #cut} with CAS. */ +protected static final AtomicReferenceFieldUpdater CONSISTENT_CUT = +AtomicReferenceFieldUpdater.newUpdater(ConsistentCutManager.class, ConsistentCut.class, "cut"); + +/** Current Consistent Cut, {@code null} if not running. */ +private volatile @Nullable ConsistentCut cut; + +/** ID of the last finished {@link ConsistentCut}. Required to avoid re-run {@link ConsistentCut} with the same id. */ +protected volatile UUID lastFinishedCutId; + +/** {@inheritDoc} */ +@Override public void start0() throws IgniteCheckedException { +super.start0(); + +cctx.exchange().registerExchangeAwareComponent
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027684007 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,212 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * ConsistentCut is a distributed algorithm that splits WAL on 2 global areas - BEFORE and AFTER. It guarantees that every + * transaction committed Before also will be committed Before on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCut} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 3. On baseline nodes: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 8. After Consistent Cut finished globally, it clears {@link ConsistentCut} variable and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** It serves updates of {@link #cut} with CAS. */ +protected static final AtomicReferenceFieldUpdater CONSISTENT_CUT = +AtomicReferenceFieldUpdater.newUpdater(ConsistentCutManager.class, ConsistentCut.class, "cut"); + +/** Current Consistent Cut, {@code null} if not running. */ +private volatile @Nullable ConsistentCut cut; + +/** ID of the last finished {@link ConsistentCut}. Required to avoid re-run {@link ConsistentCut} with the same id. */ +protected volatile UUID lastFinishedCutId; + +/** {@inheritDoc} */ +@Override public void start0() throws IgniteCheckedException { +super.start0(); + +cctx.exchange().registerExchangeAwareComponent
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027684007 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,212 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * ConsistentCut is a distributed algorithm that splits WAL on 2 global areas - BEFORE and AFTER. It guarantees that every + * transaction committed Before also will be committed Before on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCut} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 3. On baseline nodes: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 8. After Consistent Cut finished globally, it clears {@link ConsistentCut} variable and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** It serves updates of {@link #cut} with CAS. */ +protected static final AtomicReferenceFieldUpdater CONSISTENT_CUT = +AtomicReferenceFieldUpdater.newUpdater(ConsistentCutManager.class, ConsistentCut.class, "cut"); + +/** Current Consistent Cut, {@code null} if not running. */ +private volatile @Nullable ConsistentCut cut; + +/** ID of the last finished {@link ConsistentCut}. Required to avoid re-run {@link ConsistentCut} with the same id. */ +protected volatile UUID lastFinishedCutId; + +/** {@inheritDoc} */ +@Override public void start0() throws IgniteCheckedException { +super.start0(); + +cctx.exchange().registerExchangeAwareComponent
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027682658 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,105 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCut; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCutManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.lang.IgniteUuid; + +/** + * {@link ConsistentCut} splits timeline on 2 global areas - BEFORE and AFTER. It guarantees that every transaction committed + * BEFORE also will be committed BEFORE on every other node. It means that an Ignite node can safely recover itself to this + * point without any coordination with other nodes. + * + * This record is written to WAL after Consistent Cut stopped analyzing transactions from {@link ConsistentCut} and + * storing them in a particular collection - {@link #before()} or {@link #after()}. + * + * It guarantees that the BEFORE side consists of: + * 1. transactions physically committed before {@link ConsistentCutStartRecord} and weren't included into {@link #after()}; + * 2. transactions physically committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and were included into {@link #before()}. + * + * It guarantees that the AFTER side consists of: + * 1. transactions physically committed before {@link ConsistentCutStartRecord} and were included into {@link #after()}; + * 2. transactions physically committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and weren't included into {@link #before()}. + * + * @see ConsistentCutManager + */ +public class ConsistentCutFinishRecord extends WALRecord { +/** ID of {@link ConsistentCut}. */ +private final UUID cutId; + +/** + * Set of transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + * to include to the BEFORE side of Consistent Cut. + */ +private final Set before; + +/** + * Set of transactions committed before {@link ConsistentCutStartRecord} to include to the AFTER side of Consistent Cut. + */ +private final Set after; + +/** */ +public ConsistentCutFinishRecord(UUID cutId, Set before, Set after) { +this.cutId = cutId; +this.before = Collections.unmodifiableSet(before); +this.after = Collections.unmodifiableSet(after); +} + +/** */ +public Set before() { +return before; +} + +/** */ +public Set after() { +return after; +} + +/** */ +public UUID cutId() { +return cutId; +} + +/** {@inheritDoc} */ +@Override public RecordType type() { +return RecordType.CONSISTENT_CUT_FINISH_RECORD; +} + +/** {@inheritDoc} */ +@Override public String toString() { +// Dump IgniteUuid as it more convenient for debug purposes than GridCacheVersion. +List txBefore = before.stream() Review Comment: Can you, please, provide an example? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027664227 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,212 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * ConsistentCut is a distributed algorithm that splits WAL on 2 global areas - BEFORE and AFTER. It guarantees that every + * transaction committed Before also will be committed Before on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCut} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 3. On baseline nodes: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 8. After Consistent Cut finished globally, it clears {@link ConsistentCut} variable and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** It serves updates of {@link #cut} with CAS. */ +protected static final AtomicReferenceFieldUpdater CONSISTENT_CUT = +AtomicReferenceFieldUpdater.newUpdater(ConsistentCutManager.class, ConsistentCut.class, "cut"); + +/** Current Consistent Cut, {@code null} if not running. */ +private volatile @Nullable ConsistentCut cut; + +/** ID of the last finished {@link ConsistentCut}. Required to avoid re-run {@link ConsistentCut} with the same id. */ +protected volatile UUID lastFinishedCutId; + +/** {@inheritDoc} */ +@Override public void start0() throws IgniteCheckedException { +super.start0(); + +cctx.exchange().registerExchangeAwareComponent
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027660852 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,212 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * ConsistentCut is a distributed algorithm that splits WAL on 2 global areas - BEFORE and AFTER. It guarantees that every + * transaction committed Before also will be committed Before on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCut} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 3. On baseline nodes: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 8. After Consistent Cut finished globally, it clears {@link ConsistentCut} variable and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** It serves updates of {@link #cut} with CAS. */ +protected static final AtomicReferenceFieldUpdater CONSISTENT_CUT = +AtomicReferenceFieldUpdater.newUpdater(ConsistentCutManager.class, ConsistentCut.class, "cut"); + +/** Current Consistent Cut, {@code null} if not running. */ +private volatile @Nullable ConsistentCut cut; + +/** ID of the last finished {@link ConsistentCut}. Required to avoid re-run {@link ConsistentCut} with the same id. */ +protected volatile UUID lastFinishedCutId; + +/** {@inheritDoc} */ +@Override public void start0() throws IgniteCheckedException { +super.start0(); + +cctx.exchange().registerExchangeAwareComponent
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027658083 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,212 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * ConsistentCut is a distributed algorithm that splits WAL on 2 global areas - BEFORE and AFTER. It guarantees that every + * transaction committed Before also will be committed Before on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCut} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 3. On baseline nodes: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 8. After Consistent Cut finished globally, it clears {@link ConsistentCut} variable and stops wrapping messages. Review Comment: typo 5 -> 6 -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027657804 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,212 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * ConsistentCut is a distributed algorithm that splits WAL on 2 global areas - BEFORE and AFTER. It guarantees that every + * transaction committed Before also will be committed Before on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCut} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 3. On baseline nodes: + *- it writes {@link ConsistentCutStartRecord} to limit amount of transactions on the AFTER side of Consistent Cut. + * After writing this record it's safe to miss transactions on the AFTER side. + *- it collects active transactions to check which side of Consistent Cut they belong to. This collection contains all + * not-committed yet transactions that are on the BEFORE side. + *- it awaits every transaction in this collection to be committed to decide which side of Consistent Cut they belong to. + *- after the all active transactions finished it finishes Consistent Cut with writing {@link ConsistentCutFinishRecord} that contains + * collection of transactions on the BEFORE and AFTER sides. + * 8. After Consistent Cut finished globally, it clears {@link ConsistentCut} variable and stops wrapping messages. + */ +public class ConsistentCutManager extends GridCacheSharedManagerAdapter implements PartitionsExchangeAware { +/** It serves updates of {@link #cut} with CAS. */ +protected static final AtomicReferenceFieldUpdater CONSISTENT_CUT = Review Comment: Why this used? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027655716 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCutManager.java: ## @@ -0,0 +1,212 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.baselineNode; + +/** + * Processes all stuff related to Consistent Cut. + * + * ConsistentCut is a distributed algorithm that splits WAL on 2 global areas - BEFORE and AFTER. It guarantees that every + * transaction committed Before also will be committed Before on every other node participated in the transaction. + * It means that an Ignite nodes can safely recover themselves to the consistent BEFORE state without any coordination with each other. + * + * The algorithm starts on Ignite node by snapshot creation command. Other nodes are notified with discovery message of snapshot + * distributed process or by transaction messages {@link ConsistentCutAwareMessage}. + * + * The algorithm consist of steps: + * 1. On receiving new Consistent Cut ID it immediately creates new {@link ConsistentCut} before processing the message. + * 2. It starts wrapping all transaction messages to {@link ConsistentCutAwareMessage}. + * 3. Every transaction holds {@link IgniteTxAdapter#cutId()} AFTER which it committed. Value of this field is defined + *at node that commits first in distributed transaction. + * 3. On baseline nodes: Review Comment: typo 3 -> 4 -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027653903 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes current Consistent Cut. + */ +public class ConsistentCut extends GridFutureAdapter { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** ID of Consistent Cut. */ +private final UUID id; + +/** Set of checked transactions belonging to the BEFORE side. */ +@GridToStringInclude +private Set beforeCut; + +/** Set of checked transactions belonging to the AFTER side. */ +@GridToStringInclude +private Set afterCut; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedActive = ConcurrentHashMap.newKeySet(); + +/** */ +ConsistentCut(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(ConsistentCut.class); +} + +/** */ +public UUID id() { +return id; +} + +/** + * Inits local Consistent Cut: prepares list of active transactions to check which side of Consistent Cut they belong to. + */ +protected void init() throws IgniteCheckedException { +walLog(new ConsistentCutStartRecord(id), false); + +beforeCut = ConcurrentHashMap.newKeySet(); +afterCut = ConcurrentHashMap.newKeySet(); + +GridCompoundFuture checkFut = new GridCompoundFuture<>(CU.boolReducer()); + +Iterator> finFutIt = cctx.tm().activeTransactions().stream() +.filter(tx -> tx.state() != ACTIVE) +.map(IgniteInternalTx::finishFuture) +.iterator(); + +// Invoke sequentially over two iterators: +// 1. iterators are weakly consistent. +// 2. we need a guarantee to handle `removedActive` after `activeTxs` to avoid missed transactions. +checkTransactions(finFutIt, checkFut); +checkTransactions(removedActive.
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027652019 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes current Consistent Cut. + */ +public class ConsistentCut extends GridFutureAdapter { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** ID of Consistent Cut. */ +private final UUID id; + +/** Set of checked transactions belonging to the BEFORE side. */ +@GridToStringInclude +private Set beforeCut; + +/** Set of checked transactions belonging to the AFTER side. */ +@GridToStringInclude +private Set afterCut; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedActive = ConcurrentHashMap.newKeySet(); + +/** */ +ConsistentCut(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(ConsistentCut.class); +} + +/** */ +public UUID id() { +return id; +} + +/** + * Inits local Consistent Cut: prepares list of active transactions to check which side of Consistent Cut they belong to. + */ +protected void init() throws IgniteCheckedException { +walLog(new ConsistentCutStartRecord(id), false); + +beforeCut = ConcurrentHashMap.newKeySet(); +afterCut = ConcurrentHashMap.newKeySet(); + +GridCompoundFuture checkFut = new GridCompoundFuture<>(CU.boolReducer()); + +Iterator> finFutIt = cctx.tm().activeTransactions().stream() +.filter(tx -> tx.state() != ACTIVE) +.map(IgniteInternalTx::finishFuture) +.iterator(); + +// Invoke sequentially over two iterators: +// 1. iterators are weakly consistent. +// 2. we need a guarantee to handle `removedActive` after `activeTxs` to avoid missed transactions. +checkTransactions(finFutIt, checkFut); +checkTransactions(removedActive.
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027648914 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes current Consistent Cut. + */ +public class ConsistentCut extends GridFutureAdapter { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** ID of Consistent Cut. */ +private final UUID id; + +/** Set of checked transactions belonging to the BEFORE side. */ +@GridToStringInclude +private Set beforeCut; + +/** Set of checked transactions belonging to the AFTER side. */ +@GridToStringInclude +private Set afterCut; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedActive = ConcurrentHashMap.newKeySet(); Review Comment: removedFromActive ? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027648672 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes current Consistent Cut. + */ +public class ConsistentCut extends GridFutureAdapter { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** ID of Consistent Cut. */ +private final UUID id; + +/** Set of checked transactions belonging to the BEFORE side. */ +@GridToStringInclude +private Set beforeCut; + +/** Set of checked transactions belonging to the AFTER side. */ +@GridToStringInclude +private Set afterCut; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedActive = ConcurrentHashMap.newKeySet(); + +/** */ +ConsistentCut(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(ConsistentCut.class); +} + +/** */ +public UUID id() { +return id; +} + +/** + * Inits local Consistent Cut: prepares list of active transactions to check which side of Consistent Cut they belong to. + */ +protected void init() throws IgniteCheckedException { +walLog(new ConsistentCutStartRecord(id), false); + +beforeCut = ConcurrentHashMap.newKeySet(); +afterCut = ConcurrentHashMap.newKeySet(); + +GridCompoundFuture checkFut = new GridCompoundFuture<>(CU.boolReducer()); + +Iterator> finFutIt = cctx.tm().activeTransactions().stream() +.filter(tx -> tx.state() != ACTIVE) +.map(IgniteInternalTx::finishFuture) +.iterator(); + +// Invoke sequentially over two iterators: +// 1. iterators are weakly consistent. +// 2. we need a guarantee to handle `removedActive` after `activeTxs` to avoid missed transactions. +checkTransactions(finFutIt, checkFut); +checkTransactions(removedActive.
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027305941 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes current Consistent Cut. + */ +public class ConsistentCut extends GridFutureAdapter { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** ID of Consistent Cut. */ +private final UUID id; + +/** Set of checked transactions belonging to the BEFORE side. */ +@GridToStringInclude +private Set beforeCut; + +/** Set of checked transactions belonging to the AFTER side. */ +@GridToStringInclude +private Set afterCut; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedActive = ConcurrentHashMap.newKeySet(); + +/** */ +ConsistentCut(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(ConsistentCut.class); +} + +/** */ +public UUID id() { +return id; +} + +/** + * Inits local Consistent Cut: prepares list of active transactions to check which side of Consistent Cut they belong to. + */ +protected void init() throws IgniteCheckedException { +walLog(new ConsistentCutStartRecord(id), false); + +beforeCut = ConcurrentHashMap.newKeySet(); +afterCut = ConcurrentHashMap.newKeySet(); + +GridCompoundFuture checkFut = new GridCompoundFuture<>(CU.boolReducer()); + +Iterator> finFutIt = cctx.tm().activeTransactions().stream() +.filter(tx -> tx.state() != ACTIVE) +.map(IgniteInternalTx::finishFuture) +.iterator(); + +// Invoke sequentially over two iterators: +// 1. iterators are weakly consistent. +// 2. we need a guarantee to handle `removedActive` after `activeTxs` to avoid missed transactions. +checkTransactions(finFutIt, checkFut); +checkTransactions(removedActive.
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027305885 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes current Consistent Cut. + */ +public class ConsistentCut extends GridFutureAdapter { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** ID of Consistent Cut. */ +private final UUID id; + +/** Set of checked transactions belonging to the BEFORE side. */ +@GridToStringInclude +private Set beforeCut; + +/** Set of checked transactions belonging to the AFTER side. */ +@GridToStringInclude +private Set afterCut; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedActive = ConcurrentHashMap.newKeySet(); + +/** */ +ConsistentCut(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(ConsistentCut.class); +} + +/** */ +public UUID id() { +return id; +} + +/** + * Inits local Consistent Cut: prepares list of active transactions to check which side of Consistent Cut they belong to. + */ +protected void init() throws IgniteCheckedException { +walLog(new ConsistentCutStartRecord(id), false); + +beforeCut = ConcurrentHashMap.newKeySet(); +afterCut = ConcurrentHashMap.newKeySet(); + +GridCompoundFuture checkFut = new GridCompoundFuture<>(CU.boolReducer()); + +Iterator> finFutIt = cctx.tm().activeTransactions().stream() +.filter(tx -> tx.state() != ACTIVE) +.map(IgniteInternalTx::finishFuture) +.iterator(); + +// Invoke sequentially over two iterators: +// 1. iterators are weakly consistent. +// 2. we need a guarantee to handle `removedActive` after `activeTxs` to avoid missed transactions. +checkTransactions(finFutIt, checkFut); +checkTransactions(removedActive.
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027305631 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes current Consistent Cut. + */ +public class ConsistentCut extends GridFutureAdapter { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** ID of Consistent Cut. */ +private final UUID id; + +/** Set of checked transactions belonging to the BEFORE side. */ +@GridToStringInclude +private Set beforeCut; + +/** Set of checked transactions belonging to the AFTER side. */ +@GridToStringInclude +private Set afterCut; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedActive = ConcurrentHashMap.newKeySet(); + +/** */ +ConsistentCut(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(ConsistentCut.class); +} + +/** */ +public UUID id() { +return id; +} + +/** + * Inits local Consistent Cut: prepares list of active transactions to check which side of Consistent Cut they belong to. + */ +protected void init() throws IgniteCheckedException { +walLog(new ConsistentCutStartRecord(id), false); + +beforeCut = ConcurrentHashMap.newKeySet(); +afterCut = ConcurrentHashMap.newKeySet(); + +GridCompoundFuture checkFut = new GridCompoundFuture<>(CU.boolReducer()); + +Iterator> finFutIt = cctx.tm().activeTransactions().stream() +.filter(tx -> tx.state() != ACTIVE) +.map(IgniteInternalTx::finishFuture) +.iterator(); + +// Invoke sequentially over two iterators: +// 1. iterators are weakly consistent. +// 2. we need a guarantee to handle `removedActive` after `activeTxs` to avoid missed transactions. +checkTransactions(finFutIt, checkFut); +checkTransactions(removedActive.
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027305483 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes current Consistent Cut. + */ +public class ConsistentCut extends GridFutureAdapter { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** ID of Consistent Cut. */ +private final UUID id; + +/** Set of checked transactions belonging to the BEFORE side. */ +@GridToStringInclude +private Set beforeCut; + +/** Set of checked transactions belonging to the AFTER side. */ +@GridToStringInclude +private Set afterCut; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedActive = ConcurrentHashMap.newKeySet(); + +/** */ +ConsistentCut(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(ConsistentCut.class); +} + +/** */ +public UUID id() { +return id; +} + +/** + * Inits local Consistent Cut: prepares list of active transactions to check which side of Consistent Cut they belong to. + */ +protected void init() throws IgniteCheckedException { +walLog(new ConsistentCutStartRecord(id), false); + +beforeCut = ConcurrentHashMap.newKeySet(); +afterCut = ConcurrentHashMap.newKeySet(); + +GridCompoundFuture checkFut = new GridCompoundFuture<>(CU.boolReducer()); + +Iterator> finFutIt = cctx.tm().activeTransactions().stream() +.filter(tx -> tx.state() != ACTIVE) +.map(IgniteInternalTx::finishFuture) +.iterator(); + +// Invoke sequentially over two iterators: +// 1. iterators are weakly consistent. +// 2. we need a guarantee to handle `removedActive` after `activeTxs` to avoid missed transactions. +checkTransactions(finFutIt, checkFut); +checkTransactions(removedActive.
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027304911 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes current Consistent Cut. + */ +public class ConsistentCut extends GridFutureAdapter { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** ID of Consistent Cut. */ +private final UUID id; + +/** Set of checked transactions belonging to the BEFORE side. */ +@GridToStringInclude +private Set beforeCut; + +/** Set of checked transactions belonging to the AFTER side. */ +@GridToStringInclude +private Set afterCut; + +/** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ +private volatile Set> removedActive = ConcurrentHashMap.newKeySet(); + +/** */ +ConsistentCut(GridCacheSharedContext cctx, UUID id) { +this.cctx = cctx; +this.id = id; + +log = cctx.logger(ConsistentCut.class); +} + +/** */ +public UUID id() { +return id; +} + +/** + * Inits local Consistent Cut: prepares list of active transactions to check which side of Consistent Cut they belong to. + */ +protected void init() throws IgniteCheckedException { +walLog(new ConsistentCutStartRecord(id), false); Review Comment: Let's just invoke wal.log here, without unnecessary `walLog` method. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027304299 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes current Consistent Cut. + */ +public class ConsistentCut extends GridFutureAdapter { Review Comment: Let's rename this to `ConsistentCutFuture` or extract `Future` object into private variable. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027304185 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutStartRecord; +import org.apache.ignite.internal.pagemem.wal.record.RolloverType; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH; +import static org.apache.ignite.transactions.TransactionState.ACTIVE; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; +import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; +import static org.apache.ignite.transactions.TransactionState.UNKNOWN; + +/** + * Describes current Consistent Cut. + */ +public class ConsistentCut extends GridFutureAdapter { +/** */ +private final GridCacheSharedContext cctx; + +/** */ +private final IgniteLogger log; + +/** ID of Consistent Cut. */ +private final UUID id; + +/** Set of checked transactions belonging to the BEFORE side. */ +@GridToStringInclude +private Set beforeCut; Review Comment: just `before`? ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/consistentcut/ConsistentCut.java: ## @@ -0,0 +1,260 @@ +/* + * 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.processors.cache.consistentcut; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.wal.record.ConsistentCutFinishRecord; +import org.apa
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027302901 ## modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java: ## @@ -147,6 +148,9 @@ /** Deadlock detection manager. */ private DeadlockDetectionManager deadlockDetectionMgr; +/** Consistent cut manager. */ +private ConsistentCutManager consistentCutMgr; Review Comment: Let's mark this with @Nullable -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027259251 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,105 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCut; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCutManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.lang.IgniteUuid; + +/** + * {@link ConsistentCut} splits timeline on 2 global areas - BEFORE and AFTER. It guarantees that every transaction committed + * BEFORE also will be committed BEFORE on every other node. It means that an Ignite node can safely recover itself to this + * point without any coordination with other nodes. + * + * This record is written to WAL after Consistent Cut stopped analyzing transactions from {@link ConsistentCut} and + * storing them in a particular collection - {@link #before()} or {@link #after()}. + * + * It guarantees that the BEFORE side consists of: + * 1. transactions physically committed before {@link ConsistentCutStartRecord} and weren't included into {@link #after()}; + * 2. transactions physically committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and were included into {@link #before()}. + * + * It guarantees that the AFTER side consists of: + * 1. transactions physically committed before {@link ConsistentCutStartRecord} and were included into {@link #after()}; + * 2. transactions physically committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and weren't included into {@link #before()}. + * + * @see ConsistentCutManager Review Comment: Let's add links to the IEP and algorithm here. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027259221 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutStartRecord.java: ## @@ -0,0 +1,62 @@ +/* + * 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.pagemem.wal.record; + +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCut; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCutManager; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * {@link ConsistentCut} splits timeline on 2 global areas - BEFORE and AFTER. It guarantees that every transaction committed + * BEFORE also will be committed BEFORE on every other node. It means that an Ignite node can safely recover itself to this + * point without any coordination with other nodes. + * + * This record is written to WAL in moment when {@link ConsistentCut} starts on a local node. + * + * Note, there is no strict guarantee for all transactions belonged to the BEFORE side to be physically committed before + * {@link ConsistentCutStartRecord}, and vice versa. This is the reason for having {@link ConsistentCutFinishRecord}. + * + * @see ConsistentCutManager Review Comment: Let's add links to the IEP and algorithm here. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027245942 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,105 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCut; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCutManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.lang.IgniteUuid; + +/** + * {@link ConsistentCut} splits timeline on 2 global areas - BEFORE and AFTER. It guarantees that every transaction committed + * BEFORE also will be committed BEFORE on every other node. It means that an Ignite node can safely recover itself to this + * point without any coordination with other nodes. + * + * This record is written to WAL after Consistent Cut stopped analyzing transactions from {@link ConsistentCut} and + * storing them in a particular collection - {@link #before()} or {@link #after()}. + * + * It guarantees that the BEFORE side consists of: + * 1. transactions physically committed before {@link ConsistentCutStartRecord} and weren't included into {@link #after()}; + * 2. transactions physically committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and were included into {@link #before()}. + * + * It guarantees that the AFTER side consists of: + * 1. transactions physically committed before {@link ConsistentCutStartRecord} and were included into {@link #after()}; + * 2. transactions physically committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and weren't included into {@link #before()}. + * + * @see ConsistentCutManager + */ +public class ConsistentCutFinishRecord extends WALRecord { +/** ID of {@link ConsistentCut}. */ +private final UUID cutId; + +/** + * Set of transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + * to include to the BEFORE side of Consistent Cut. + */ +private final Set before; + +/** + * Set of transactions committed before {@link ConsistentCutStartRecord} to include to the AFTER side of Consistent Cut. + */ +private final Set after; + +/** */ +public ConsistentCutFinishRecord(UUID cutId, Set before, Set after) { +this.cutId = cutId; +this.before = Collections.unmodifiableSet(before); +this.after = Collections.unmodifiableSet(after); +} + +/** */ +public Set before() { +return before; +} + +/** */ +public Set after() { +return after; +} + +/** */ +public UUID cutId() { +return cutId; +} + +/** {@inheritDoc} */ +@Override public RecordType type() { +return RecordType.CONSISTENT_CUT_FINISH_RECORD; +} + +/** {@inheritDoc} */ +@Override public String toString() { +// Dump IgniteUuid as it more convenient for debug purposes than GridCacheVersion. +List txBefore = before.stream() +.map(GridCacheVersion::asIgniteUuid) +.collect(Collectors.toList()); + +List txAfter = after.stream() +.map(GridCacheVersion::asIgniteUuid) +.collect(Collectors.toList()); + +return "ConsistentCutFinishRecord [before=" + txBefore + ", after=" + txAfter + "]"; Review Comment: Let's use GridToStringBuilder. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027245918 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,105 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCut; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCutManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.lang.IgniteUuid; + +/** + * {@link ConsistentCut} splits timeline on 2 global areas - BEFORE and AFTER. It guarantees that every transaction committed + * BEFORE also will be committed BEFORE on every other node. It means that an Ignite node can safely recover itself to this + * point without any coordination with other nodes. + * + * This record is written to WAL after Consistent Cut stopped analyzing transactions from {@link ConsistentCut} and + * storing them in a particular collection - {@link #before()} or {@link #after()}. + * + * It guarantees that the BEFORE side consists of: + * 1. transactions physically committed before {@link ConsistentCutStartRecord} and weren't included into {@link #after()}; + * 2. transactions physically committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and were included into {@link #before()}. + * + * It guarantees that the AFTER side consists of: + * 1. transactions physically committed before {@link ConsistentCutStartRecord} and were included into {@link #after()}; + * 2. transactions physically committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and weren't included into {@link #before()}. + * + * @see ConsistentCutManager + */ +public class ConsistentCutFinishRecord extends WALRecord { +/** ID of {@link ConsistentCut}. */ +private final UUID cutId; + +/** + * Set of transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + * to include to the BEFORE side of Consistent Cut. + */ +private final Set before; + +/** + * Set of transactions committed before {@link ConsistentCutStartRecord} to include to the AFTER side of Consistent Cut. + */ +private final Set after; + +/** */ +public ConsistentCutFinishRecord(UUID cutId, Set before, Set after) { +this.cutId = cutId; +this.before = Collections.unmodifiableSet(before); +this.after = Collections.unmodifiableSet(after); +} + +/** */ +public Set before() { +return before; +} + +/** */ +public Set after() { +return after; +} + +/** */ +public UUID cutId() { +return cutId; +} + +/** {@inheritDoc} */ +@Override public RecordType type() { +return RecordType.CONSISTENT_CUT_FINISH_RECORD; +} + +/** {@inheritDoc} */ +@Override public String toString() { +// Dump IgniteUuid as it more convenient for debug purposes than GridCacheVersion. +List txBefore = before.stream() Review Comment: Why we need to map sets to `IgniteUuid`? Can, you, provide examples how this will simplify debug or operations? -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027245222 ## modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ConsistentCutFinishRecord.java: ## @@ -0,0 +1,105 @@ +/* + * 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.pagemem.wal.record; + +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCut; +import org.apache.ignite.internal.processors.cache.consistentcut.ConsistentCutManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.lang.IgniteUuid; + +/** + * {@link ConsistentCut} splits timeline on 2 global areas - BEFORE and AFTER. It guarantees that every transaction committed + * BEFORE also will be committed BEFORE on every other node. It means that an Ignite node can safely recover itself to this + * point without any coordination with other nodes. + * + * This record is written to WAL after Consistent Cut stopped analyzing transactions from {@link ConsistentCut} and + * storing them in a particular collection - {@link #before()} or {@link #after()}. + * + * It guarantees that the BEFORE side consists of: + * 1. transactions physically committed before {@link ConsistentCutStartRecord} and weren't included into {@link #after()}; + * 2. transactions physically committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and were included into {@link #before()}. + * + * It guarantees that the AFTER side consists of: + * 1. transactions physically committed before {@link ConsistentCutStartRecord} and were included into {@link #after()}; + * 2. transactions physically committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + *and weren't included into {@link #before()}. + * + * @see ConsistentCutManager + */ +public class ConsistentCutFinishRecord extends WALRecord { +/** ID of {@link ConsistentCut}. */ +private final UUID cutId; + +/** + * Set of transactions committed between {@link ConsistentCutStartRecord} and {@link ConsistentCutFinishRecord} + * to include to the BEFORE side of Consistent Cut. + */ +private final Set before; + +/** + * Set of transactions committed before {@link ConsistentCutStartRecord} to include to the AFTER side of Consistent Cut. + */ +private final Set after; + +/** */ +public ConsistentCutFinishRecord(UUID cutId, Set before, Set after) { +this.cutId = cutId; +this.before = Collections.unmodifiableSet(before); Review Comment: Please, remove wrapping objects. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [ignite] nizhikov commented on a diff in pull request #10314: IGNITE-17029 Consistent Cut for incremental snapshot
nizhikov commented on code in PR #10314: URL: https://github.com/apache/ignite/pull/10314#discussion_r1027242045 ## modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java: ## @@ -189,21 +189,23 @@ public static void main(String[] args) throws Exception { //gen.generateAndWrite(GridNearLockResponse.class); //gen.generateAndWrite(GridDhtLockRequest.class); //gen.generateAndWrite(GridDhtLockResponse.class); -// Review Comment: Please, revert unncessary changes. -- 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. To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org