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<WALPointer> { + /** 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<GridCacheVersion> before; + + /** Set of checked transactions belong to the AFTER set. */ + @GridToStringInclude + private Set<GridCacheVersion> after; + + /** Collection of transactions removed from {@link IgniteTxManager#activeTransactions()}. */ + private volatile Set<IgniteInternalFuture<IgniteInternalTx>> 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<Boolean, Boolean> checkFut = new GridCompoundFuture<>(CU.boolReducer()); + + Iterator<IgniteInternalFuture<IgniteInternalTx>> 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<IgniteInternalFuture<IgniteInternalTx>> removedFromActiveIter = removedFromActive.iterator(); + removedFromActive = null; + checkTransactions(removedFromActiveIter, checkFut); + + checkFut.markInitialized(); + + checkFut.listen(finish -> { + if (isDone()) + return; + + if (Boolean.FALSE.equals(finish.result())) { + onDone(new IgniteCheckedException("Cut is inconsistent.")); + + return; + } + + cctx.database().checkpointReadLock(); + + try { + WALPointer ptr = cctx.wal().log(new ConsistentCutFinishRecord(id, before, after), RolloverType.CURRENT_SEGMENT); + + onDone(ptr); Review Comment: Do we really need to hold `checkpointReadLock` while `onDone` invoked? -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
