From 5f71d4acd7e84cd2506c017e61e6d875e3634c7c Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 9 Oct 2024 22:36:36 +0300 Subject: [PATCH 01/27] first --- .../snapshot/IgniteSnapshotManager.java | 8 +- .../snapshot/SnapshotCheckProcess.java | 5 +- .../snapshot/SnapshotCheckProcessRequest.java | 12 ++ .../SnapshotPartitionsVerifyTask.java | 125 ------------------ .../IgniteClusterSnapshotCheckTest.java | 50 ++----- 5 files changed, 32 insertions(+), 168 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java index 2d16637af6e99..fc4a1906104b3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java @@ -1904,8 +1904,8 @@ public IgniteInternalFuture checkSnapshot( ", incIdx=" + incIdx + ", grps=" + grps + ", validateParts=" + check + ']'); } - if (check && (incIdx < 1 || !includeCustomHandlers)) - return checkSnpProc.start(name, snpPath, grps, incIdx, includeCustomHandlers); + if (incIdx < 1 || !includeCustomHandlers) + return checkSnpProc.start(name, snpPath, grps, check, incIdx, includeCustomHandlers); GridFutureAdapter res = new GridFutureAdapter<>(); @@ -1928,9 +1928,7 @@ public IgniteInternalFuture checkSnapshot( if (f0.error() == null && F.isEmpty(metasRes.exceptions())) { Map> metas = metasRes.meta(); - Class cls = includeCustomHandlers - ? SnapshotHandlerRestoreTask.class - : SnapshotPartitionsVerifyTask.class; + Class cls = SnapshotHandlerRestoreTask.class; kctx0.task().execute( cls, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index e9f64247e82d6..ef3fc04ff9030 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -221,7 +221,7 @@ private IgniteInternalFuture validateParts(SnapshotCheckP workingFut = req.allRestoreHandlers() ? snpMgr.checker().invokeCustomHandlers(ctx.locMeta, req.snapshotPath(), req.groups(), true) : snpMgr.checker().checkPartitions(ctx.locMeta, snpMgr.snapshotLocalDir(req.snapshotName(), req.snapshotPath()), - req.groups(), false, true, false); + req.groups(), false, req.fullCheck(), false); } workingFut.whenComplete((res, err) -> { @@ -387,6 +387,7 @@ private void reducePreparationAndMetasCheck( * @param snpName Snapshot name. * @param snpPath Snapshot directory path. * @param grpNames List of cache group names. + * @param fullCheck If {@code true}, calculates partition hashes. Otherwise, checks only snapshot integrity and partition counters. * @param incIdx Incremental snapshot index. If not positive, snapshot is not considered as incremental. * @param allRestoreHandlers If {@code true}, all the registered {@link IgniteSnapshotManager#handlers()} of type * {@link SnapshotHandlerType#RESTORE} are invoked. Otherwise, only snapshot metadatas and partition @@ -396,6 +397,7 @@ public IgniteInternalFuture start( String snpName, @Nullable String snpPath, @Nullable Collection grpNames, + boolean fullCheck, int incIdx, boolean allRestoreHandlers ) { @@ -414,6 +416,7 @@ public IgniteInternalFuture start( snpName, snpPath, grpNames, + fullCheck, incIdx, allRestoreHandlers ); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java index d1b38d4d51266..4b0123c254688 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java @@ -33,6 +33,10 @@ public class SnapshotCheckProcessRequest extends AbstractSnapshotOperationReques /** Serial version uid. */ private static final long serialVersionUID = 0L; + /** If {@code true}, calculates partition hashes. Otherwise, checks only snapshot integrity and partition counters. */ + @GridToStringInclude + private final boolean fullCheck; + /** * If {@code true}, all the registered {@link IgniteSnapshotManager#handlers()} of type {@link SnapshotHandlerType#RESTORE} * are invoked. Otherwise, only snapshot metadatas and partition hashes are validated. @@ -52,6 +56,7 @@ public class SnapshotCheckProcessRequest extends AbstractSnapshotOperationReques * @param nodes Baseline node IDs that must be alive to complete the operation.. * @param snpPath Snapshot directory path. * @param grps List of cache group names. + * @param fullCheck If {@code true}, calculates partition hashes. Otherwise, checks only snapshot integrity and partition counters. * @param incIdx Incremental snapshot index. If not positive, snapshot is not considered as incremental. * @param allRestoreHandlers If {@code true}, all the registered {@link IgniteSnapshotManager#handlers()} of type * {@link SnapshotHandlerType#RESTORE} are invoked. Otherwise, only snapshot metadatas and @@ -63,6 +68,7 @@ public class SnapshotCheckProcessRequest extends AbstractSnapshotOperationReques String snpName, String snpPath, @Nullable Collection grps, + boolean fullCheck, int incIdx, boolean allRestoreHandlers ) { @@ -70,6 +76,7 @@ public class SnapshotCheckProcessRequest extends AbstractSnapshotOperationReques assert !F.isEmpty(nodes); + this.fullCheck = fullCheck; this.allRestoreHandlers = allRestoreHandlers; this.incIdx = incIdx; } @@ -82,6 +89,11 @@ public boolean allRestoreHandlers() { return allRestoreHandlers; } + /** If {@code true}, calculates partition hashes. Otherwise, checks only snapshot integrity and partition counters. */ + public boolean fullCheck() { + return fullCheck; + } + /** @return Incremental snapshot index. If not positive, snapshot is not considered as incremental. */ public int incrementalIndex() { return incIdx; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java deleted file mode 100644 index 1ad30d2c981cb..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java +++ /dev/null @@ -1,125 +0,0 @@ -/* - * 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.io.File; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import org.apache.ignite.IgniteException; -import org.apache.ignite.compute.ComputeJobResult; -import org.apache.ignite.internal.management.cache.PartitionKeyV2; -import org.apache.ignite.internal.management.cache.VerifyBackupPartitionsTaskV2; -import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; -import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecordV2; -import org.apache.ignite.internal.processors.task.GridInternal; -import org.jetbrains.annotations.Nullable; - -import static org.apache.ignite.internal.management.cache.VerifyBackupPartitionsTaskV2.reduce0; - -/** - * Task for checking snapshot partitions consistency the same way as {@link VerifyBackupPartitionsTaskV2} does. - * Since a snapshot partitions already stored apart on disk the is no requirement for a cluster upcoming updates - * to be hold on. - */ -@GridInternal -public class SnapshotPartitionsVerifyTask extends AbstractSnapshotVerificationTask { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** {@inheritDoc} */ - @Override protected VerifySnapshotPartitionsJob createJob(String name, String consId, SnapshotPartitionsVerifyTaskArg args) { - return new VerifySnapshotPartitionsJob(name, args.snapshotPath(), consId, args.cacheGroupNames(), args.check()); - } - - /** {@inheritDoc} */ - @Override public @Nullable SnapshotPartitionsVerifyTaskResult reduce(List results) throws IgniteException { - return new SnapshotPartitionsVerifyTaskResult(metas, reduce0(results)); - } - - /** Job that collects update counters of snapshot partitions on the node it executes. */ - private static class VerifySnapshotPartitionsJob extends AbstractSnapshotVerificationJob { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** - * @param snpName Snapshot name to validate. - * @param consId Consistent id of the related node. - * @param rqGrps Set of cache groups to be checked in the snapshot or {@code empty} to check everything. - * @param snpPath Snapshot directory path. - * @param check If {@code true} check snapshot before restore. - */ - public VerifySnapshotPartitionsJob( - String snpName, - @Nullable String snpPath, - String consId, - Collection rqGrps, - boolean check - ) { - super(snpName, snpPath, consId, rqGrps, check); - } - - /** {@inheritDoc} */ - @Override public Map execute() throws IgniteException { - GridCacheSharedContext cctx = ignite.context().cache().context(); - - if (log.isInfoEnabled()) { - log.info("Verify snapshot partitions procedure has been initiated " + - "[snpName=" + snpName + ", consId=" + consId + ']'); - } - - File snpDir = cctx.snapshotMgr().snapshotLocalDir(snpName, snpPath); - - try { - SnapshotMetadata meta = cctx.snapshotMgr().readSnapshotMetadata(snpDir, consId); - - return cctx.snapshotMgr().checker().checkPartitions(meta, snpDir, rqGrps, false, check, false).get(); - } - catch (Exception e) { - throw new IgniteException("Failed to read snapshot metadatas of the snapshot '" + snpName + "'.", e); - } - finally { - if (log.isInfoEnabled()) { - log.info("Verify snapshot partitions procedure has been finished " + - "[snpName=" + snpName + ", consId=" + consId + ']'); - } - } - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - - if (o == null || getClass() != o.getClass()) - return false; - - VerifySnapshotPartitionsJob job = (VerifySnapshotPartitionsJob)o; - - return snpName.equals(job.snpName) && consId.equals(job.consId) && - Objects.equals(rqGrps, job.rqGrps) && Objects.equals(snpPath, job.snpPath) && - Objects.equals(check, job.check); - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - return Objects.hash(snpName, consId, rqGrps, snpPath, check); - } - } -} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java index ac041c439e5cf..d5bee923563c6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java @@ -52,7 +52,6 @@ import org.apache.ignite.compute.ComputeJobResult; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.GridJobExecuteRequest; import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; @@ -86,6 +85,7 @@ import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.distributed.FullMessage; +import org.apache.ignite.internal.util.distributed.SingleNodeMessage; import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.util.lang.GridIterator; import org.apache.ignite.internal.util.typedef.F; @@ -341,17 +341,18 @@ public void testClusterSnapshotCheckOtherCluster() throws Exception { Set assigns = Collections.newSetFromMap(new ConcurrentHashMap<>()); for (int i = 4; i < 7; i++) { - startGrid(optimize(getConfiguration(getTestIgniteInstanceName(i)).setCacheConfiguration())); + IgniteEx grid = startGrid(optimize(getConfiguration(getTestIgniteInstanceName(i)).setCacheConfiguration())); - UUID locNodeId = grid(i).localNode().id(); + if (!U.isLocalNodeCoordinator(grid.context().discovery())) + continue; - grid(i).context().io().addMessageListener(GridTopic.TOPIC_JOB, new GridMessageListener() { + grid.context().io().addMessageListener(GridTopic.TOPIC_DISTRIBUTED_PROCESS, new GridMessageListener() { @Override public void onMessage(UUID nodeId, Object msg, byte plc) { - if (msg instanceof GridJobExecuteRequest) { - GridJobExecuteRequest msg0 = (GridJobExecuteRequest)msg; + if (msg instanceof SingleNodeMessage) { + SingleNodeMessage msg0 = (SingleNodeMessage)msg; - if (msg0.getTaskName().contains(SnapshotPartitionsVerifyTask.class.getName())) - assigns.add(locNodeId); + if (msg0.type() == CHECK_SNAPSHOT_PARTS.ordinal()) + assigns.add(nodeId); } } }); @@ -361,14 +362,14 @@ public void testClusterSnapshotCheckOtherCluster() throws Exception { ignite.cluster().baselineAutoAdjustEnabled(false); ignite.cluster().state(ACTIVE); - IdleVerifyResultV2 res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null, null, false, 0, false).get().idleVerifyResult(); + IdleVerifyResultV2 res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); res.print(b::append, true); // GridJobExecuteRequest is not send to the local node. - assertTrue("Number of jobs must be equal to the cluster size (except local node): " + assigns + ", count: " - + assigns.size(), waitForCondition(() -> assigns.size() == 2, 5_000L)); + assertTrue("Number of distributed process single messages must be equal to the cluster size: " + + assigns + ", count: " + assigns.size(), waitForCondition(() -> assigns.size() == 2, 5_000L)); assertTrue(F.isEmpty(res.exceptions())); assertPartitionsSame(res); @@ -522,20 +523,7 @@ public void testClusterSnapshotCheckHashesSameAsIdleVerifyHashes() throws Except IdleVerifyResultV2 idleVerifyRes = ignite.compute().execute(new TestVisorBackupPartitionsTask(), arg); - IdleVerifyResultV2 snpVerifyRes = ignite.compute().execute( - new TestSnapshotPartitionsVerifyTask(), - new SnapshotPartitionsVerifyTaskArg( - new HashSet<>(), - Collections.singletonMap(ignite.cluster().localNode(), - Collections.singletonList(snp(ignite).readSnapshotMetadata( - snp(ignite).snapshotLocalDir(SNAPSHOT_NAME), - (String)ignite.configuration().getConsistentId() - ))), - null, - 0, - true - ) - ).idleVerifyResult(); + IdleVerifyResultV2 snpVerifyRes = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); Map> idleVerifyHashes = jobResults.get(TestVisorBackupPartitionsTask.class); Map> snpCheckHashes = jobResults.get(TestVisorBackupPartitionsTask.class); @@ -1427,16 +1415,4 @@ private class TestVisorBackupPartitionsTask extends VerifyBackupPartitionsTaskV2 return res; } } - - /** Test compute task to collect partition data hashes when the snapshot check procedure ends. */ - private class TestSnapshotPartitionsVerifyTask extends SnapshotPartitionsVerifyTask { - /** {@inheritDoc} */ - @Override public @Nullable SnapshotPartitionsVerifyTaskResult reduce(List results) throws IgniteException { - SnapshotPartitionsVerifyTaskResult res = super.reduce(results); - - saveHashes(TestSnapshotPartitionsVerifyTask.class, results); - - return res; - } - } } From 6291dc904bb13d28aa79888904b0dc5e834e8aa0 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Thu, 10 Oct 2024 15:28:13 +0300 Subject: [PATCH 02/27] raw --- .../common/ComputeTaskPermissionsTest.java | 4 +- .../ignite/util/GridCommandHandlerTest.java | 4 +- .../snapshot/SnapshotCheckCommand.java | 6 +- .../snapshot/SnapshotCheckTask.java | 10 +- .../AbstractSnapshotVerificationTask.java | 156 ------------------ .../snapshot/IgniteSnapshotManager.java | 64 +------ .../snapshot/SnapshotCheckProcess.java | 16 +- .../snapshot/SnapshotHandlerRestoreTask.java | 102 ------------ .../SnapshotMetadataVerificationTask.java | 122 -------------- .../SnapshotMetadataVerificationTaskArg.java | 118 ------------- ...napshotMetadataVerificationTaskResult.java | 75 --------- ...va => SnapshotPartitionsVerifyResult.java} | 6 +- .../SnapshotPartitionsVerifyTaskArg.java | 134 --------------- .../resources/META-INF/classnames.properties | 9 - .../IgniteClusterSnapshotCheckTest.java | 20 +-- .../IgniteClusterSnapshotStreamerTest.java | 6 +- .../ConcurrentTxsIncrementalSnapshotTest.java | 4 +- ...ementalSnapshotCheckBeforeRestoreTest.java | 10 +- 18 files changed, 52 insertions(+), 814 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotVerificationTask.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTaskArg.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTaskResult.java rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{SnapshotPartitionsVerifyTaskResult.java => SnapshotPartitionsVerifyResult.java} (96%) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTaskArg.java diff --git a/modules/clients/src/test/java/org/apache/ignite/common/ComputeTaskPermissionsTest.java b/modules/clients/src/test/java/org/apache/ignite/common/ComputeTaskPermissionsTest.java index e3e337468acae..3f06b638a355d 100644 --- a/modules/clients/src/test/java/org/apache/ignite/common/ComputeTaskPermissionsTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/common/ComputeTaskPermissionsTest.java @@ -51,7 +51,7 @@ import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.client.GridClientConfiguration; import org.apache.ignite.internal.client.GridClientFactory; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadataVerificationTask; +import org.apache.ignite.internal.management.cache.IdleVerifyTaskV2; import org.apache.ignite.internal.processors.security.AbstractSecurityTest; import org.apache.ignite.internal.processors.security.OperationSecurityContext; import org.apache.ignite.internal.processors.security.PublicAccessJob; @@ -112,7 +112,7 @@ public class ComputeTaskPermissionsTest extends AbstractSecurityTest { private static final IgniteClosure SYSTEM_CLOSURE = new ToStringClosure<>(); /** */ - private static final ComputeTask SYSTEM_TASK = new SnapshotMetadataVerificationTask(); + private static final ComputeTask SYSTEM_TASK = new IdleVerifyTaskV2(); /** */ private static final AtomicInteger EXECUTED_TASK_CNTR = new AtomicInteger(); diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index 51dd7ff9629f1..21fadddc76fdc 100644 --- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -104,7 +104,7 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.snapshot.DataStreamerUpdatesHandler; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyTaskResult; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyResult; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyImpl; @@ -3347,7 +3347,7 @@ public void testCheckSnapshot() throws Exception { StringBuilder sb = new StringBuilder(); - ((SnapshotPartitionsVerifyTaskResult)h.getLastOperationResult()).print(sb::append); + ((SnapshotPartitionsVerifyResult)h.getLastOperationResult()).print(sb::append); assertContains(log, sb.toString(), "The check procedure has finished, no conflicts have been found"); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckCommand.java index 11f833819c0b4..3bccad3d2251f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckCommand.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckCommand.java @@ -18,10 +18,10 @@ package org.apache.ignite.internal.management.snapshot; import java.util.function.Consumer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyTaskResult; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyResult; /** */ -public class SnapshotCheckCommand extends AbstractSnapshotCommand { +public class SnapshotCheckCommand extends AbstractSnapshotCommand { /** {@inheritDoc} */ @Override public String description() { return "Check snapshot"; @@ -38,7 +38,7 @@ public class SnapshotCheckCommand extends AbstractSnapshotCommand printer) { + @Override public void printResult(SnapshotCheckCommandArg arg, SnapshotPartitionsVerifyResult res, Consumer printer) { res.print(printer); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java index 0d48dac64ccd5..73742c866e9b5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java @@ -19,7 +19,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyTaskResult; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyResult; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.visor.VisorJob; @@ -29,17 +29,17 @@ * @see IgniteSnapshotManager#checkSnapshot(String, String) */ @GridInternal -public class SnapshotCheckTask extends VisorOneNodeTask { +public class SnapshotCheckTask extends VisorOneNodeTask { /** Serial version uid. */ private static final long serialVersionUID = 0L; /** {@inheritDoc} */ - @Override protected VisorJob job(SnapshotCheckCommandArg arg) { + @Override protected VisorJob job(SnapshotCheckCommandArg arg) { return new SnapshotCheckJob(arg, debug); } /** */ - private static class SnapshotCheckJob extends SnapshotJob { + private static class SnapshotCheckJob extends SnapshotJob { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -52,7 +52,7 @@ protected SnapshotCheckJob(SnapshotCheckCommandArg arg, boolean debug) { } /** {@inheritDoc} */ - @Override protected SnapshotPartitionsVerifyTaskResult run(SnapshotCheckCommandArg arg) throws IgniteException { + @Override protected SnapshotPartitionsVerifyResult run(SnapshotCheckCommandArg arg) throws IgniteException { IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); return new IgniteFutureImpl<>(snpMgr.checkSnapshot(arg.snapshotName(), arg.src(), arg.increment())).get(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotVerificationTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotVerificationTask.java deleted file mode 100644 index e8d348fc6da5d..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotVerificationTask.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * 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.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.ignite.IgniteException; -import org.apache.ignite.IgniteLogger; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.compute.ComputeJob; -import org.apache.ignite.compute.ComputeJobAdapter; -import org.apache.ignite.compute.ComputeJobResult; -import org.apache.ignite.compute.ComputeJobResultPolicy; -import org.apache.ignite.compute.ComputeTaskAdapter; -import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.resources.IgniteInstanceResource; -import org.apache.ignite.resources.LoggerResource; -import org.jetbrains.annotations.Nullable; - -/** - * The task for checking the consistency of snapshots in the cluster. - */ -public abstract class AbstractSnapshotVerificationTask extends - ComputeTaskAdapter { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Map of snapshot metadata information found on each cluster node. */ - protected final Map> metas = new HashMap<>(); - - /** Ignite instance. */ - @IgniteInstanceResource - protected IgniteEx ignite; - - /** Injected logger. */ - @LoggerResource - protected IgniteLogger log; - - /** {@inheritDoc} */ - @Override public Map map(List subgrid, SnapshotPartitionsVerifyTaskArg arg) { - Map> clusterMetas = arg.clusterMetadata(); - - if (!subgrid.containsAll(clusterMetas.keySet())) { - throw new IgniteSnapshotVerifyException(F.asMap(ignite.localNode(), - new IgniteException("Some of Ignite nodes left the cluster during the snapshot verification " + - "[curr=" + F.viewReadOnly(subgrid, F.node2id()) + - ", init=" + F.viewReadOnly(clusterMetas.keySet(), F.node2id()) + ']'))); - } - - Map jobs = new HashMap<>(); - Set allMetas = new HashSet<>(); - clusterMetas.values().forEach(allMetas::addAll); - metas.putAll(clusterMetas); - - while (!allMetas.isEmpty()) { - for (Map.Entry> e : clusterMetas.entrySet()) { - SnapshotMetadata meta = F.find(e.getValue(), null, allMetas::remove); - - if (meta == null) - continue; - - jobs.put(createJob(meta.snapshotName(), meta.consistentId(), arg), e.getKey()); - - if (allMetas.isEmpty()) - break; - } - } - - return jobs; - } - - /** {@inheritDoc} */ - @Override public ComputeJobResultPolicy result(ComputeJobResult res, List rcvd) throws IgniteException { - // Handle all exceptions during the `reduce` operation. - return ComputeJobResultPolicy.WAIT; - } - - /** - * @param name Snapshot name. - * @param consId Consistent id of the related node. - * @param args Check snapshot parameters. - * - * @return Compute job. - */ - protected abstract AbstractSnapshotVerificationJob createJob(String name, String consId, SnapshotPartitionsVerifyTaskArg args); - - /** */ - protected abstract static class AbstractSnapshotVerificationJob extends ComputeJobAdapter { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Ignite instance. */ - @IgniteInstanceResource - protected IgniteEx ignite; - - /** Injected logger. */ - @LoggerResource - protected IgniteLogger log; - - /** Snapshot name. */ - protected final String snpName; - - /** Snapshot directory path. */ - @Nullable protected final String snpPath; - - /** Consistent id of the related node. */ - protected final String consId; - - /** Set of cache groups to be checked in the snapshot. {@code Null} or empty to check everything. */ - @Nullable protected final Collection rqGrps; - - /** If {@code true}, calculates and compares partition hashes. Otherwise, only basic snapshot validation is launched. */ - protected final boolean check; - - /** - * @param snpName Snapshot name. - * @param snpPath Snapshot directory path. - * @param consId Consistent id of the related node. - * @param rqGrps Set of cache groups to be checked in the snapshot. {@code Null} or empty to check everything. - * @param check If {@code true}, calculates and compares partition hashes. Otherwise, only basic snapshot validation is launched. - */ - protected AbstractSnapshotVerificationJob( - String snpName, - @Nullable String snpPath, - String consId, - @Nullable Collection rqGrps, - boolean check - ) { - this.snpName = snpName; - this.snpPath = snpPath; - this.consId = consId; - this.rqGrps = rqGrps; - this.check = check; - } - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java index fc4a1906104b3..f83ce0e595021 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java @@ -1842,7 +1842,7 @@ public IgniteFuture cancelLocalRestoreTask(String name) { * @return Future with the result of execution snapshot partitions verify task, which besides calculating partition * hashes of {@link IdleVerifyResultV2} also contains the snapshot metadata distribution across the cluster. */ - public IgniteInternalFuture checkSnapshot(String name, @Nullable String snpPath) { + public IgniteInternalFuture checkSnapshot(String name, @Nullable String snpPath) { return checkSnapshot(name, snpPath, -1); } @@ -1855,7 +1855,7 @@ public IgniteInternalFuture checkSnapshot(St * @return Future with the result of execution snapshot partitions verify task, which besides calculating partition * hashes of {@link IdleVerifyResultV2} also contains the snapshot metadata distribution across the cluster. */ - public IgniteInternalFuture checkSnapshot(String name, @Nullable String snpPath, int incIdx) { + public IgniteInternalFuture checkSnapshot(String name, @Nullable String snpPath, int incIdx) { A.notNullOrEmpty(name, "Snapshot name cannot be null or empty."); A.ensure(U.alphanumericUnderscore(name), "Snapshot name must satisfy the following name pattern: a-zA-Z0-9_"); @@ -1886,7 +1886,7 @@ public IgniteInternalFuture checkSnapshot(St * @return Future with the result of execution snapshot partitions verify task, which besides calculating partition * hashes of {@link IdleVerifyResultV2} also contains the snapshot metadata distribution across the cluster. */ - public IgniteInternalFuture checkSnapshot( + public IgniteInternalFuture checkSnapshot( String name, @Nullable String snpPath, @Nullable Collection grps, @@ -1904,62 +1904,16 @@ public IgniteInternalFuture checkSnapshot( ", incIdx=" + incIdx + ", grps=" + grps + ", validateParts=" + check + ']'); } - if (incIdx < 1 || !includeCustomHandlers) - return checkSnpProc.start(name, snpPath, grps, check, incIdx, includeCustomHandlers); + IgniteInternalFuture res = checkSnpProc.start(name, snpPath, grps, check, incIdx, + includeCustomHandlers); - GridFutureAdapter res = new GridFutureAdapter<>(); - - GridKernalContext kctx0 = cctx.kernalContext(); - - Collection bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE), - (node) -> CU.baselineNode(node, kctx0.state().clusterState())); - - Collection grpIds = grps == null ? Collections.emptySet() : F.viewReadOnly(grps, CU::cacheId); - - SnapshotMetadataVerificationTaskArg taskArg = new SnapshotMetadataVerificationTaskArg(name, snpPath, incIdx, grpIds); - - kctx0.task().execute( - SnapshotMetadataVerificationTask.class, - taskArg, - options(bltNodes) - ).listen(f0 -> { - SnapshotMetadataVerificationTaskResult metasRes = f0.result(); - - if (f0.error() == null && F.isEmpty(metasRes.exceptions())) { - Map> metas = metasRes.meta(); - - Class cls = SnapshotHandlerRestoreTask.class; - - kctx0.task().execute( - cls, - new SnapshotPartitionsVerifyTaskArg(grps, metas, snpPath, incIdx, check), - options(new ArrayList<>(metas.keySet())) - ).listen(f1 -> { - if (f1.error() == null) - res.onDone(f1.result()); - else if (f1.error() instanceof IgniteSnapshotVerifyException) - res.onDone(new SnapshotPartitionsVerifyTaskResult(metas, - new IdleVerifyResultV2(((IgniteSnapshotVerifyException)f1.error()).exceptions()))); - else - res.onDone(f1.error()); - }); - } - else { - if (f0.error() == null) - res.onDone(new IgniteSnapshotVerifyException(metasRes.exceptions())); - else if (f0.error() instanceof IgniteSnapshotVerifyException) - res.onDone(new SnapshotPartitionsVerifyTaskResult(null, - new IdleVerifyResultV2(((IgniteSnapshotVerifyException)f0.error()).exceptions()))); - else - res.onDone(f0.error()); + res.listen(lsnr -> { + if (log.isInfoEnabled()) { + log.info("The check snapshot procedure finished [snpName=" + name + ", snpPath=" + snpPath + + ", incIdx=" + incIdx + ", grps=" + grps + ']'); } }); - if (log.isInfoEnabled()) { - res.listen(() -> log.info("The check snapshot procedure finished [snpName=" + name + - ", snpPath=" + snpPath + ", incIdx=" + incIdx + ", grps=" + grps + ']')); - } - return res; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index ef3fc04ff9030..8e3daa4048522 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -65,7 +65,7 @@ public class SnapshotCheckProcess { private final Map contexts = new ConcurrentHashMap<>(); /** Cluster-wide operation futures per snapshot called from current node. */ - private final Map> clusterOpFuts = new ConcurrentHashMap<>(); + private final Map> clusterOpFuts = new ConcurrentHashMap<>(); /** Check metas first phase subprocess. */ private final DistributedProcess phase1CheckMetas; @@ -141,7 +141,7 @@ private IgniteInternalFuture reduceValidatePartsAndFinish( if (log.isInfoEnabled()) log.info("Finished snapshot validation [req=" + ctx.req + ']'); - GridFutureAdapter clusterOpFut = clusterOpFuts.get(reqId); + GridFutureAdapter clusterOpFut = clusterOpFuts.get(reqId); if (clusterOpFut == null) return new GridFinishedFuture<>(); @@ -156,7 +156,7 @@ private IgniteInternalFuture reduceValidatePartsAndFinish( mapErrors(errors) ); - clusterOpFut.onDone(new SnapshotPartitionsVerifyTaskResult(ctx.clusterMetas, chkRes)); + clusterOpFut.onDone(new SnapshotPartitionsVerifyResult(ctx.clusterMetas, chkRes)); } else if (ctx.req.allRestoreHandlers()) { try { @@ -168,7 +168,7 @@ else if (ctx.req.allRestoreHandlers()) { checker.checkCustomHandlersResults(ctx.req.snapshotName(), cstRes); - clusterOpFut.onDone(new SnapshotPartitionsVerifyTaskResult(ctx.clusterMetas, null)); + clusterOpFut.onDone(new SnapshotPartitionsVerifyResult(ctx.clusterMetas, null)); } catch (Throwable err) { clusterOpFut.onDone(err); @@ -183,7 +183,7 @@ else if (ctx.req.allRestoreHandlers()) { IdleVerifyResultV2 chkRes = SnapshotChecker.reduceHashesResults(results0, errors0); - clusterOpFut.onDone(new SnapshotPartitionsVerifyTaskResult(ctx.clusterMetas, chkRes)); + clusterOpFut.onDone(new SnapshotPartitionsVerifyResult(ctx.clusterMetas, chkRes)); } else clusterOpFut.onDone(new IgniteSnapshotVerifyException(errors0)); @@ -325,7 +325,7 @@ private void reducePreparationAndMetasCheck( SnapshotCheckContext ctx = context(null, reqId); // The context is not stored in the case of concurrent check of the same snapshot but the operation future is registered. - GridFutureAdapter clusterOpFut = clusterOpFuts.get(reqId); + GridFutureAdapter clusterOpFut = clusterOpFuts.get(reqId); try { if (!errors.isEmpty()) @@ -393,7 +393,7 @@ private void reducePreparationAndMetasCheck( * {@link SnapshotHandlerType#RESTORE} are invoked. Otherwise, only snapshot metadatas and partition * hashes are validated. */ - public IgniteInternalFuture start( + public IgniteInternalFuture start( String snpName, @Nullable String snpPath, @Nullable Collection grpNames, @@ -421,7 +421,7 @@ public IgniteInternalFuture start( allRestoreHandlers ); - GridFutureAdapter clusterOpFut = new GridFutureAdapter<>(); + GridFutureAdapter clusterOpFut = new GridFutureAdapter<>(); clusterOpFut.listen(fut -> { clusterOpFuts.remove(reqId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java deleted file mode 100644 index 7df4a551b752a..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * 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.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.ignite.IgniteException; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.compute.ComputeJobResult; -import org.apache.ignite.internal.util.typedef.F; -import org.jetbrains.annotations.Nullable; - -/** - * Snapshot restore operation handling task. - */ -public class SnapshotHandlerRestoreTask extends AbstractSnapshotVerificationTask { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** {@inheritDoc} */ - @Override protected SnapshotHandlerRestoreJob createJob(String name, String consId, SnapshotPartitionsVerifyTaskArg args) { - return new SnapshotHandlerRestoreJob(name, args.snapshotPath(), consId, args.cacheGroupNames(), args.check()); - } - - /** {@inheritDoc} */ - @SuppressWarnings("rawtypes") - @Nullable @Override public SnapshotPartitionsVerifyTaskResult reduce(List results) { - String snpName = F.first(F.first(metas.values())).snapshotName(); - - Map>> resMap = new HashMap<>(); - - results.forEach(jobRes -> { - if (jobRes.getException() != null) - throw jobRes.getException(); - else - resMap.put(jobRes.getNode(), jobRes.getData()); - }); - - try { - ignite.context().cache().context().snapshotMgr().checker().checkCustomHandlersResults(snpName, resMap); - } - catch (Exception e) { - log.warning("The snapshot operation will be aborted due to a handler error [snapshot=" + snpName + "].", e); - - throw new IgniteException(e); - } - - return new SnapshotPartitionsVerifyTaskResult(metas, null); - } - - /** Invokes all {@link SnapshotHandlerType#RESTORE} handlers locally. */ - private static class SnapshotHandlerRestoreJob extends AbstractSnapshotVerificationJob { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** - * @param snpName Snapshot name. - * @param snpPath Snapshot directory path. - * @param consId Consistent id of the related node. - * @param grps Cache group names. - * @param check If {@code true} check snapshot before restore. - */ - public SnapshotHandlerRestoreJob( - String snpName, - @Nullable String snpPath, - String consId, - Collection grps, - boolean check - ) { - super(snpName, snpPath, consId, grps, check); - } - - /** {@inheritDoc} */ - @Override public Map> execute() { - try { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); - - return snpMgr.checker().invokeCustomHandlers(snpName, consId, snpPath, rqGrps, check).get(); - } - catch (Exception e) { - throw new IgniteException("Filed to invoke all the snapshot validation handlers.", e); - } - } - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java deleted file mode 100644 index bad025849b317..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.ignite.IgniteException; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.compute.ComputeJob; -import org.apache.ignite.compute.ComputeJobAdapter; -import org.apache.ignite.compute.ComputeJobResult; -import org.apache.ignite.compute.ComputeJobResultPolicy; -import org.apache.ignite.compute.ComputeTaskAdapter; -import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.processors.task.GridInternal; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.resources.IgniteInstanceResource; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -/** Snapshot task to verify snapshot metadata on the baseline nodes for given snapshot name. */ -@GridInternal -public class SnapshotMetadataVerificationTask - extends ComputeTaskAdapter { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** */ - private SnapshotMetadataVerificationTaskArg arg; - - /** */ - @IgniteInstanceResource - private transient IgniteEx ignite; - - /** {@inheritDoc} */ - @Override public @NotNull Map map( - List subgrid, - SnapshotMetadataVerificationTaskArg arg - ) throws IgniteException { - this.arg = arg; - - Map map = U.newHashMap(subgrid.size()); - - for (ClusterNode node : subgrid) - map.put(new MetadataVerificationJob(arg), node); - - return map; - } - - /** Job that verifies snapshot on an Ignite node. */ - private static class MetadataVerificationJob extends ComputeJobAdapter { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - @IgniteInstanceResource - private transient IgniteEx ignite; - - /** */ - private final SnapshotMetadataVerificationTaskArg arg; - - /** */ - public MetadataVerificationJob(SnapshotMetadataVerificationTaskArg arg) { - this.arg = arg; - } - - /** {@inheritDoc} */ - @Override public List execute() { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); - - return snpMgr.checker().checkLocalMetas(snpMgr.snapshotLocalDir(arg.snapshotName(), arg.snapshotPath()), - arg.incrementIndex(), arg.grpIds(), ignite.localNode().consistentId()).join(); - } - } - - /** {@inheritDoc} */ - @Override public @Nullable SnapshotMetadataVerificationTaskResult reduce( - List results) throws IgniteException { - Map> reduceRes = new HashMap<>(); - Map exs = new HashMap<>(); - - for (ComputeJobResult res : results) { - if (res.getException() != null) { - exs.put(res.getNode(), res.getException()); - - continue; - } - - if (!F.isEmpty((Collection)res.getData())) - reduceRes.computeIfAbsent(res.getNode(), n -> new ArrayList<>()).addAll(res.getData()); - } - - exs = SnapshotChecker.reduceMetasResults(arg.snapshotName(), arg.snapshotPath(), reduceRes, exs, ignite.localNode().consistentId()); - - return new SnapshotMetadataVerificationTaskResult(reduceRes, exs); - } - - /** {@inheritDoc} */ - @Override public ComputeJobResultPolicy result(ComputeJobResult res, List rcvd) throws IgniteException { - // Handle all exceptions during the `reduce` operation. - return ComputeJobResultPolicy.WAIT; - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTaskArg.java deleted file mode 100644 index 6bc69b2c7d305..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTaskArg.java +++ /dev/null @@ -1,118 +0,0 @@ -/* - * 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.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.Collection; -import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.internal.visor.VisorDataTransferObject; -import org.jetbrains.annotations.Nullable; - -/** - * Input parameters for checking snapshot metadata. - */ -public class SnapshotMetadataVerificationTaskArg extends VisorDataTransferObject { - /** Serial version UID. */ - private static final long serialVersionUID = 0L; - - /** Snapshot name. */ - private String snpName; - - /** Snapshot directory path. */ - private String snpPath; - - /** Incremental snapshot index. */ - private int incIdx; - - /** Cache group ids. */ - @Nullable private Collection grpIds; - - /** Default constructor. */ - public SnapshotMetadataVerificationTaskArg() { - // No-op. - } - - /** - * @param snpName Snapshot name. - * @param snpPath Snapshot directory path. - */ - public SnapshotMetadataVerificationTaskArg(String snpName, @Nullable String snpPath, int incIdx, Collection grpIds) { - this.snpName = snpName; - this.snpPath = snpPath; - this.incIdx = incIdx; - this.grpIds = grpIds; - } - - /** {@inheritDoc} */ - @Override public byte getProtocolVersion() { - return V2; - } - - /** - * @return Snapshot name. - */ - public String snapshotName() { - return snpName; - } - - /** - * @return Snapshot directory path. - */ - public String snapshotPath() { - return snpPath; - } - - /** - * @return Incremental snapshot index. - */ - public int incrementIndex() { - return incIdx; - } - - /** - * @return Cache group ids. - */ - @Nullable public Collection grpIds() { - return grpIds; - } - - /** {@inheritDoc} */ - @Override protected void writeExternalData(ObjectOutput out) throws IOException { - U.writeString(out, snpName); - U.writeString(out, snpPath); - out.writeInt(incIdx); - U.writeCollection(out, grpIds); - } - - /** {@inheritDoc} */ - @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { - snpName = U.readString(in); - snpPath = U.readString(in); - incIdx = in.readInt(); - - if (protoVer > V1) - grpIds = U.readCollection(in); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(SnapshotMetadataVerificationTaskArg.class, this); - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTaskResult.java deleted file mode 100644 index 212f615ce1a8e..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTaskResult.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.dto.IgniteDataTransferObject; -import org.apache.ignite.internal.util.typedef.internal.U; - -/** */ -public class SnapshotMetadataVerificationTaskResult extends IgniteDataTransferObject { - /** */ - private static final long serialVersionUID = 0L; - - /** Full snapshot metadata. */ - private Map> meta; - - /** Errors happened during snapshot metadata verification. */ - private Map exceptions; - - /** */ - public SnapshotMetadataVerificationTaskResult( - Map> meta, - Map exceptions - ) { - this.meta = Collections.unmodifiableMap(meta); - this.exceptions = Collections.unmodifiableMap(exceptions); - } - - /** */ - public SnapshotMetadataVerificationTaskResult() { - } - - /** @return Errors happened during snapshot metadata verification. */ - public Map exceptions() { - return Collections.unmodifiableMap(exceptions); - } - - /** @return Full snapshot metadata. */ - public Map> meta() { - return Collections.unmodifiableMap(meta); - } - - /** {@inheritDoc} */ - @Override protected void writeExternalData(ObjectOutput out) throws IOException { - U.writeMap(out, meta); - U.writeMap(out, exceptions); - } - - /** {@inheritDoc} */ - @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { - meta = U.readMap(in); - exceptions = U.readMap(in); - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyResult.java similarity index 96% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTaskResult.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyResult.java index a32397742ecbb..560f6237c1c67 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTaskResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyResult.java @@ -38,7 +38,7 @@ * The result of execution snapshot partitions verify task which besides calculating partition hashes of * {@link IdleVerifyResultV2} also contains the snapshot metadata distribution across the cluster. */ -public class SnapshotPartitionsVerifyTaskResult extends IgniteDataTransferObject { +public class SnapshotPartitionsVerifyResult extends IgniteDataTransferObject { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -49,7 +49,7 @@ public class SnapshotPartitionsVerifyTaskResult extends IgniteDataTransferObject @Nullable private IdleVerifyResultV2 idleRes; /** Default constructor. */ - public SnapshotPartitionsVerifyTaskResult() { + public SnapshotPartitionsVerifyResult() { // No-op. } @@ -57,7 +57,7 @@ public SnapshotPartitionsVerifyTaskResult() { * @param metas Map of snapshot metadata information found on each cluster node. * @param idleRes Result of cluster nodes partitions comparison. */ - public SnapshotPartitionsVerifyTaskResult( + SnapshotPartitionsVerifyResult( Map> metas, @Nullable IdleVerifyResultV2 idleRes ) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTaskArg.java deleted file mode 100644 index c3c660d94b5a3..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTaskArg.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * 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.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.internal.visor.VisorDataTransferObject; -import org.jetbrains.annotations.Nullable; - -/** - * Input parameters for checking snapshot partitions consistency task. - */ -public class SnapshotPartitionsVerifyTaskArg extends VisorDataTransferObject { - /** Serial version UID. */ - private static final long serialVersionUID = 0L; - - /** Cache group names to be verified. */ - @Nullable private Collection grpNames; - - /** The map of distribution of snapshot metadata pieces across the cluster. */ - private Map> clusterMetas; - - /** Snapshot directory path. */ - @Nullable private String snpPath; - - /** If {@code true} check snapshot integrity. */ - private boolean check; - - /** Incremental snapshot index. */ - private int incIdx; - - /** Default constructor. */ - public SnapshotPartitionsVerifyTaskArg() { - // No-op. - } - - /** - * @param grpNames Cache group names to be verified. - * @param clusterMetas The map of distribution of snapshot metadata pieces across the cluster. - * @param snpPath Snapshot directory path. - * @param incIdx Incremental snapshot index. - * @param check If {@code true} check snapshot integrity. - */ - public SnapshotPartitionsVerifyTaskArg( - @Nullable Collection grpNames, - Map> clusterMetas, - @Nullable String snpPath, - int incIdx, - boolean check - ) { - this.grpNames = grpNames; - this.clusterMetas = clusterMetas; - this.snpPath = snpPath; - this.incIdx = incIdx; - this.check = check; - } - - /** - * @return Cache group names to be verified. - */ - @Nullable public Collection cacheGroupNames() { - return grpNames; - } - - /** - * @return The map of distribution of snapshot metadata pieces across the cluster. - */ - public Map> clusterMetadata() { - return clusterMetas; - } - - /** - * @return Snapshot directory path. - */ - @Nullable public String snapshotPath() { - return snpPath; - } - - /** - * @return Incremental snapshot index. - */ - public int incrementIndex() { - return incIdx; - } - - /** @return If {@code true} check snapshot integrity. */ - public boolean check() { - return check; - } - - /** {@inheritDoc} */ - @Override protected void writeExternalData(ObjectOutput out) throws IOException { - U.writeCollection(out, grpNames); - U.writeMap(out, clusterMetas); - U.writeString(out, snpPath); - out.writeBoolean(check); - out.writeInt(incIdx); - } - - /** {@inheritDoc} */ - @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { - grpNames = U.readCollection(in); - clusterMetas = U.readMap(in); - snpPath = U.readString(in); - check = in.readBoolean(); - incIdx = in.readInt(); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(SnapshotPartitionsVerifyTaskArg.class, this); - } -} diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index 63d8fd49495ba..f892a1f21c5b8 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -1144,7 +1144,6 @@ org.apache.ignite.internal.processors.cache.persistence.migration.UpgradePending org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl$Segment org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl$ThrottlingPolicy org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotMessage -org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotVerificationTask org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager$3 org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager$CancelSnapshotCallable org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager$CreateSnapshotCallable @@ -1166,16 +1165,8 @@ org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotHandler org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotHandlerResult org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotHandlerType org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadata -org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadataVerificationTask -org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadataVerificationTask$MetadataVerificationJob -org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadataVerificationTaskArg -org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadataVerificationTaskResult org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperation org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationRequest -org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyTask -org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyTask$VerifySnapshotPartitionsJob -org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyTaskArg -org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyTaskResult org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreProcess$SnapshotRestoreOperationResponse org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreStatusTask org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreStatusTask$1 diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java index d5bee923563c6..0e6fbf134c170 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java @@ -538,7 +538,7 @@ public void testClusterSnapshotCheckHashesSameAsIdleVerifyHashes() throws Except /** @throws Exception If fails. */ @Test public void testClusterSnapshotCheckWithTwoCachesCheckNullInput() throws Exception { - SnapshotPartitionsVerifyTaskResult res = checkSnapshotWithTwoCachesWhenOneIsCorrupted(null); + SnapshotPartitionsVerifyResult res = checkSnapshotWithTwoCachesWhenOneIsCorrupted(null); StringBuilder b = new StringBuilder(); res.idleVerifyResult().print(b::append, true); @@ -552,7 +552,7 @@ public void testClusterSnapshotCheckWithTwoCachesCheckNullInput() throws Excepti /** @throws Exception If fails. */ @Test public void testClusterSnapshotCheckWithTwoCachesCheckNotCorrupted() throws Exception { - SnapshotPartitionsVerifyTaskResult res = checkSnapshotWithTwoCachesWhenOneIsCorrupted(Collections.singletonList( + SnapshotPartitionsVerifyResult res = checkSnapshotWithTwoCachesWhenOneIsCorrupted(Collections.singletonList( OPTIONAL_CACHE_NAME)); StringBuilder b = new StringBuilder(); @@ -567,7 +567,7 @@ public void testClusterSnapshotCheckWithTwoCachesCheckNotCorrupted() throws Exce /** @throws Exception If fails. */ @Test public void testClusterSnapshotCheckWithTwoCachesCheckTwoCaches() throws Exception { - SnapshotPartitionsVerifyTaskResult res = checkSnapshotWithTwoCachesWhenOneIsCorrupted(Arrays.asList( + SnapshotPartitionsVerifyResult res = checkSnapshotWithTwoCachesWhenOneIsCorrupted(Arrays.asList( OPTIONAL_CACHE_NAME, DEFAULT_CACHE_NAME)); StringBuilder b = new StringBuilder(); @@ -778,7 +778,7 @@ public void testConcurrentDifferentSnpFullChecksAllowed() throws Exception { } } - /** Tests that concurrent snapshot full check and restoration (without checking) are allowed for different snapshots. */ + /** Tests that concurrent snapshot full check and restoration (without full checking) are allowed for different snapshots. */ @Test public void testConcurrentDifferentSnpFullCheckAndRestorationAllowed() throws Exception { prepareGridsAndSnapshot(3, 2, 2, false); @@ -890,9 +890,9 @@ public void testConcurrentTheSameSnpFullCheckWhenFullyRestoringDeclined() throws } } - /** Tests that concurrent full check and restoration (without checking) of the same snapshot are allowed. */ + /** Tests that concurrent full check and restoration (without full checking) of the same snapshot are declined. */ @Test - public void testConcurrentTheSameSnpFullCheckAndRestoreAllowed() throws Exception { + public void testConcurrentTheSameSnpFullCheckAndRestoreDeclined() throws Exception { prepareGridsAndSnapshot(3, 2, 2, true); for (int i = 0; i < G.allGrids().size(); ++i) { @@ -905,8 +905,8 @@ public void testConcurrentTheSameSnpFullCheckAndRestoreAllowed() throws Exceptio () -> new IgniteFutureImpl<>(snp(grid(i0)).checkSnapshot(SNAPSHOT_NAME, null)), () -> snp(grid(j0)).restoreSnapshot(SNAPSHOT_NAME, null), CHECK_SNAPSHOT_METAS, - RESTORE_CACHE_GROUP_SNAPSHOT_START, - false, + CHECK_SNAPSHOT_METAS, + true, false, null, () -> grid(0).destroyCache(DEFAULT_CACHE_NAME) @@ -1314,7 +1314,7 @@ public void testClusterSnapshotCheckWithExpiring() throws Exception { snp(ignite).createSnapshot(SNAPSHOT_NAME).get(timeout); - SnapshotPartitionsVerifyTaskResult res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get(timeout); + SnapshotPartitionsVerifyResult res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get(timeout); assertFalse(res.idleVerifyResult().hasConflicts()); } @@ -1344,7 +1344,7 @@ private void saveHashes(Class cls, List results) { * @return Check result. * @throws Exception If fails. */ - private SnapshotPartitionsVerifyTaskResult checkSnapshotWithTwoCachesWhenOneIsCorrupted( + private SnapshotPartitionsVerifyResult checkSnapshotWithTwoCachesWhenOneIsCorrupted( Collection cachesToCheck ) throws Exception { Random rnd = new Random(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotStreamerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotStreamerTest.java index 2d0b0e65bb419..fbec40d362dc1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotStreamerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotStreamerTest.java @@ -351,7 +351,7 @@ private void doTestDataStreamerWhileSnapshot(IgniteEx snpHnd, boolean allowOverw String notExpWrn = allowOverwrite ? null : SnapshotPartitionsQuickVerifyHandler.WRN_MSG; try { - SnapshotPartitionsVerifyTaskResult checkRes = createAndCheckSnapshot(snpHnd, true, expectedWrn, + SnapshotPartitionsVerifyResult checkRes = createAndCheckSnapshot(snpHnd, true, expectedWrn, notExpWrn); if (expectedWrn != null) { @@ -462,7 +462,7 @@ private IgniteInternalFuture runLoad(Ignite ldr, boolean allowOverwrite, Atom } /** */ - private SnapshotPartitionsVerifyTaskResult createAndCheckSnapshot(IgniteEx snpHnd, boolean create, + private SnapshotPartitionsVerifyResult createAndCheckSnapshot(IgniteEx snpHnd, boolean create, String expWrn, String notExpWrn) throws Exception { assert notExpWrn == null || expWrn != null; @@ -498,7 +498,7 @@ private SnapshotPartitionsVerifyTaskResult createAndCheckSnapshot(IgniteEx snpHn } } - SnapshotPartitionsVerifyTaskResult checkRes = snp(snpHnd).checkSnapshot(SNAPSHOT_NAME, null).get(); + SnapshotPartitionsVerifyResult checkRes = snp(snpHnd).checkSnapshot(SNAPSHOT_NAME, null).get(); assertTrue(checkRes.exceptions().isEmpty()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/ConcurrentTxsIncrementalSnapshotTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/ConcurrentTxsIncrementalSnapshotTest.java index e2fadff5dc036..1501554ddc67b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/ConcurrentTxsIncrementalSnapshotTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/ConcurrentTxsIncrementalSnapshotTest.java @@ -32,7 +32,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyTaskResult; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyResult; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.transactions.Transaction; @@ -267,7 +267,7 @@ private void testConcurrentTransactionsAndSnapshots(Supplier tx) throws checkWalsConsistency(txCnt.get(), SNP_CNT); for (int i = 0; i < SNP_CNT; i++) { - SnapshotPartitionsVerifyTaskResult res = snp(grid(0)).checkSnapshot(SNP, null, i).get(getTestTimeout()); + SnapshotPartitionsVerifyResult res = snp(grid(0)).checkSnapshot(SNP, null, i).get(getTestTimeout()); assertTrue(F.isEmpty(res.exceptions())); assertFalse(res.idleVerifyResult().hasConflicts()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java index ad94d4010653a..8723234381629 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java @@ -29,7 +29,7 @@ import org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotSelfTest; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IncrementalSnapshotMetadata; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyTaskResult; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyResult; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -97,7 +97,7 @@ public void testCheckCorrectIncrementalSnapshot() throws Exception { for (IgniteEx n: F.asList(grid(0), grid(GRID_CNT))) { for (int i = 0; i <= incSnpCnt; i++) { - SnapshotPartitionsVerifyTaskResult res = snp(n).checkSnapshot(SNP, null, null, false, i, DFLT_CHECK_ON_RESTORE) + SnapshotPartitionsVerifyResult res = snp(n).checkSnapshot(SNP, null, null, false, i, DFLT_CHECK_ON_RESTORE) .get(getTestTimeout()); assertTrue(res.exceptions().isEmpty()); @@ -123,7 +123,7 @@ public void testNonExistentIncrementalSnapshot() throws Exception { createIncrementalSnapshots(1); for (IgniteEx n : F.asList(grid(0), grid(GRID_CNT))) { - SnapshotPartitionsVerifyTaskResult res = snp(n).checkSnapshot(SNP, null, null, false, 1, DFLT_CHECK_ON_RESTORE) + SnapshotPartitionsVerifyResult res = snp(n).checkSnapshot(SNP, null, null, false, 1, DFLT_CHECK_ON_RESTORE) .get(getTestTimeout()); assertTrue(res.exceptions().isEmpty()); @@ -160,7 +160,7 @@ public void testIntermediateSnapshotNotFound() throws Exception { U.delete(snp(srv).incrementalSnapshotLocalDir(SNP, null, 1)); for (IgniteEx n : F.asList(srv, grid(GRID_CNT))) { - SnapshotPartitionsVerifyTaskResult res = snp(n).checkSnapshot(SNP, null, null, false, 0, DFLT_CHECK_ON_RESTORE) + SnapshotPartitionsVerifyResult res = snp(n).checkSnapshot(SNP, null, null, false, 0, DFLT_CHECK_ON_RESTORE) .get(getTestTimeout()); assertTrue(res.exceptions().isEmpty()); @@ -188,7 +188,7 @@ public void testWalSegmentsNotFound() throws Exception { deleteWalSegment(0); for (IgniteEx n : F.asList(srv, grid(GRID_CNT))) { - SnapshotPartitionsVerifyTaskResult res = snp(n).checkSnapshot(SNP, null, null, false, 0, DFLT_CHECK_ON_RESTORE) + SnapshotPartitionsVerifyResult res = snp(n).checkSnapshot(SNP, null, null, false, 0, DFLT_CHECK_ON_RESTORE) .get(getTestTimeout()); assertTrue(res.exceptions().isEmpty()); From a34a1561435ce25d9d81f268d00dba38f5b01ab3 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Thu, 10 Oct 2024 15:52:50 +0300 Subject: [PATCH 03/27] minor javadoc fix --- .../cache/persistence/snapshot/SnapshotCheckProcess.java | 3 ++- .../persistence/snapshot/SnapshotCheckProcessRequest.java | 7 ++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index 8e3daa4048522..7022cb94cf8ed 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -387,7 +387,8 @@ private void reducePreparationAndMetasCheck( * @param snpName Snapshot name. * @param snpPath Snapshot directory path. * @param grpNames List of cache group names. - * @param fullCheck If {@code true}, calculates partition hashes. Otherwise, checks only snapshot integrity and partition counters. + * @param fullCheck If {@code true}, additionally calculates partition hashes. Otherwise, checks only snapshot integrity + * and partition counters. * @param incIdx Incremental snapshot index. If not positive, snapshot is not considered as incremental. * @param allRestoreHandlers If {@code true}, all the registered {@link IgniteSnapshotManager#handlers()} of type * {@link SnapshotHandlerType#RESTORE} are invoked. Otherwise, only snapshot metadatas and partition diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java index 4b0123c254688..3fa731fc0e468 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java @@ -33,7 +33,7 @@ public class SnapshotCheckProcessRequest extends AbstractSnapshotOperationReques /** Serial version uid. */ private static final long serialVersionUID = 0L; - /** If {@code true}, calculates partition hashes. Otherwise, checks only snapshot integrity and partition counters. */ + /** If {@code true}, additionally calculates partition hashes. Otherwise, checks only snapshot integrity and partition counters. */ @GridToStringInclude private final boolean fullCheck; @@ -56,7 +56,8 @@ public class SnapshotCheckProcessRequest extends AbstractSnapshotOperationReques * @param nodes Baseline node IDs that must be alive to complete the operation.. * @param snpPath Snapshot directory path. * @param grps List of cache group names. - * @param fullCheck If {@code true}, calculates partition hashes. Otherwise, checks only snapshot integrity and partition counters. + * @param fullCheck If {@code true}, additionally calculates partition hashes. Otherwise, checks only snapshot integrity + * and partition counters. * @param incIdx Incremental snapshot index. If not positive, snapshot is not considered as incremental. * @param allRestoreHandlers If {@code true}, all the registered {@link IgniteSnapshotManager#handlers()} of type * {@link SnapshotHandlerType#RESTORE} are invoked. Otherwise, only snapshot metadatas and @@ -89,7 +90,7 @@ public boolean allRestoreHandlers() { return allRestoreHandlers; } - /** If {@code true}, calculates partition hashes. Otherwise, checks only snapshot integrity and partition counters. */ + /** If {@code true}, additionally calculates partition hashes. Otherwise, checks only snapshot integrity and partition counters. */ public boolean fullCheck() { return fullCheck; } From f0d09c04e5640d2c14722c8a4efcbd98eca0c397 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Thu, 10 Oct 2024 18:30:42 +0300 Subject: [PATCH 04/27] fix --- .../cache/persistence/snapshot/SnapshotRestoreProcess.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java index c02deccd103e1..a0ea3bbb01fc7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java @@ -333,7 +333,7 @@ public IgniteFutureImpl start( snpMgr.recordSnapshotEvent(snpName, msg, EventType.EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED); - snpMgr.checkSnapshot(snpName, snpPath, cacheGrpNames, true, incIdx, check).listen(f -> { + snpMgr.checkSnapshot(snpName, snpPath, cacheGrpNames, incIdx < 1, incIdx, check).listen(f -> { if (f.error() != null) { finishProcess(fut0.rqId, f.error()); From a9f640103995e186094ef7598f81287a87db4d84 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Thu, 10 Oct 2024 19:17:28 +0300 Subject: [PATCH 05/27] test fixes --- .../apache/ignite/common/ComputeTaskPermissionsTest.java | 4 ++-- .../cache/persistence/snapshot/EncryptedSnapshotTest.java | 4 +++- .../persistence/snapshot/IgniteSnapshotMXBeanTest.java | 7 +++++-- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/common/ComputeTaskPermissionsTest.java b/modules/clients/src/test/java/org/apache/ignite/common/ComputeTaskPermissionsTest.java index 3f06b638a355d..c5a6116b9683a 100644 --- a/modules/clients/src/test/java/org/apache/ignite/common/ComputeTaskPermissionsTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/common/ComputeTaskPermissionsTest.java @@ -51,7 +51,7 @@ import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.client.GridClientConfiguration; import org.apache.ignite.internal.client.GridClientFactory; -import org.apache.ignite.internal.management.cache.IdleVerifyTaskV2; +import org.apache.ignite.internal.processors.cache.verify.CollectConflictPartitionKeysTask; import org.apache.ignite.internal.processors.security.AbstractSecurityTest; import org.apache.ignite.internal.processors.security.OperationSecurityContext; import org.apache.ignite.internal.processors.security.PublicAccessJob; @@ -112,7 +112,7 @@ public class ComputeTaskPermissionsTest extends AbstractSecurityTest { private static final IgniteClosure SYSTEM_CLOSURE = new ToStringClosure<>(); /** */ - private static final ComputeTask SYSTEM_TASK = new IdleVerifyTaskV2(); + private static final ComputeTask SYSTEM_TASK = new CollectConflictPartitionKeysTask(); /** */ private static final AtomicInteger EXECUTED_TASK_CNTR = new AtomicInteger(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java index b1a719687e793..86f30620b90bb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java @@ -43,6 +43,7 @@ import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_SNAPSHOT_DIRECTORY; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.MASTER_KEY_CHANGE_PREPARE; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE; import static org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause; /** @@ -387,7 +388,8 @@ private void checkActionFailsDuringSnapshotOperation(boolean restore, Function msg instanceof FullMessage && ((FullMessage)msg).error().isEmpty()); + spi0.block((msg) -> msg instanceof FullMessage && ((FullMessage)msg).error().isEmpty() + && ((FullMessage)msg).type() == RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE.ordinal()); fut = grid(1).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singletonList(dfltCacheCfg.getName())); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java index 91b88a45461b0..d1a804ada61c7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java @@ -40,6 +40,7 @@ import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.SNAPSHOT_METRICS; import static org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreProcess.SNAPSHOT_RESTORE_METRICS; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_START; import static org.apache.ignite.testframework.GridTestUtils.assertContains; import static org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause; import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; @@ -262,7 +263,8 @@ public void testStatus() throws Exception { awaitPartitionMapExchange(); - spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage); + spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage + && ((SingleNodeMessage)msg).type() == RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE.ordinal()); fut = srv.snapshot().restoreSnapshot(SNAPSHOT_NAME, F.asList(DEFAULT_CACHE_NAME)); @@ -283,7 +285,8 @@ public void testStatus() throws Exception { awaitPartitionMapExchange(); - spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage); + spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage + && ((SingleNodeMessage)msg).type() == RESTORE_CACHE_GROUP_SNAPSHOT_START.ordinal()); fut = srv.snapshot().restoreSnapshot(SNAPSHOT_NAME, F.asList(DEFAULT_CACHE_NAME), 1); From 6b331957647330478df41d69041f29f96d0a1459 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Fri, 11 Oct 2024 09:13:20 +0300 Subject: [PATCH 06/27] test fixes --- .../org/apache/ignite/util/GridCommandHandlerTest.java | 7 +++++-- .../cache/persistence/snapshot/SnapshotChecker.java | 7 ++++--- .../snapshot/IgniteClusterSnapshotRestoreSelfTest.java | 2 +- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index 21fadddc76fdc..ae6d68b2bb866 100644 --- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -173,6 +173,7 @@ import static org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreProcess.SNAPSHOT_RESTORE_METRICS; import static org.apache.ignite.internal.processors.cache.verify.IdleVerifyUtility.GRID_NOT_IDLE_MSG; import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.DEFAULT_TARGET_FOLDER; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE; import static org.apache.ignite.testframework.GridTestUtils.assertContains; import static org.apache.ignite.testframework.GridTestUtils.assertNotContains; import static org.apache.ignite.testframework.GridTestUtils.assertThrows; @@ -3832,7 +3833,8 @@ public void testSnapshotStatus() throws Exception { awaitPartitionMapExchange(); - spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage); + spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage + && ((SingleNodeMessage)msg).type() == RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE.ordinal()); fut = srv.snapshot().restoreSnapshot(snapshotName, F.asList(DEFAULT_CACHE_NAME)); @@ -3851,7 +3853,8 @@ public void testSnapshotStatus() throws Exception { awaitPartitionMapExchange(); - spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage); + spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage + && ((SingleNodeMessage)msg).type() == RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE.ordinal()); fut = srv.snapshot().restoreSnapshot(snapshotName, F.asList(DEFAULT_CACHE_NAME), 1); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java index 8a12db38ffdf8..994a362221f46 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java @@ -354,7 +354,7 @@ public CompletableFuture checkIncrementalSnapsho return CompletableFuture.supplyAsync( () -> { - String consId = kctx.cluster().get().localNode().consistentId().toString(); + Object consId = kctx.cluster().get().localNode().consistentId(); File snpDir = kctx.cache().context().snapshotMgr().snapshotLocalDir(snpName, snpPath); @@ -366,9 +366,10 @@ public CompletableFuture checkIncrementalSnapsho BaselineTopology blt = kctx.state().clusterState().baselineTopology(); - SnapshotMetadata meta = kctx.cache().context().snapshotMgr().readSnapshotMetadata(snpDir, consId); + SnapshotMetadata meta = kctx.cache().context().snapshotMgr().readSnapshotMetadata(snpDir, consId.toString()); - if (!F.eqNotOrdered(blt.consistentIds(), meta.baselineNodes())) { + if (!F.eqNotOrdered(blt.consistentIds().stream().map(Object::toString).collect(Collectors.toList()), + meta.baselineNodes())) { throw new IgniteCheckedException("Topologies of snapshot and current cluster are different [snp=" + meta.baselineNodes() + ", current=" + blt.consistentIds() + ']'); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java index 3fd9d6dde406f..05a0e254c6ac9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java @@ -280,7 +280,7 @@ private void checkStartClusterSnapshotRestoreMultithreaded(IntSupplier nodeIdxSu } catch (Exception e) { assertTrue("Unexpected exception: " + Throwables.getStackTraceAsString(e), - X.hasCause(e, "The previous snapshot restore operation was not completed.", + X.hasCause(e, "Validation of snapshot '" + SNAPSHOT_NAME + "' has already started", IgniteCheckedException.class, IgniteException.class)); failCnt.incrementAndGet(); From e9fabb40ed1cf1d319e4e16bd28887be60e4bcde Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Fri, 11 Oct 2024 15:26:48 +0300 Subject: [PATCH 07/27] test fix --- .../snapshot/IgniteClusterSnapshotRestoreSelfTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java index 05a0e254c6ac9..335078fc7abf8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java @@ -280,8 +280,8 @@ private void checkStartClusterSnapshotRestoreMultithreaded(IntSupplier nodeIdxSu } catch (Exception e) { assertTrue("Unexpected exception: " + Throwables.getStackTraceAsString(e), - X.hasCause(e, "Validation of snapshot '" + SNAPSHOT_NAME + "' has already started", - IgniteCheckedException.class, IgniteException.class)); + X.hasCause(e, "The previous snapshot restore operation was not completed", IgniteException.class) + || X.hasCause(e, "has already started", IgniteException.class)); failCnt.incrementAndGet(); } From ffd33bff2cbf8e04820bb5b9804bdd1f5745e70a Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Sat, 12 Oct 2024 20:45:27 +0300 Subject: [PATCH 08/27] raw --- .../snapshot/SnapshotCheckProcess.java | 85 ++++++++++++++----- 1 file changed, 63 insertions(+), 22 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index 7022cb94cf8ed..ec270bc9d871a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -19,6 +19,7 @@ import java.io.Serializable; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -163,8 +164,9 @@ else if (ctx.req.allRestoreHandlers()) { if (!errors.isEmpty()) throw F.firstValue(errors); - Map>> cstRes = mapResults(results, ctx.req.nodes(), - SnapshotCheckResponse::customHandlersResults); +// Map>> cstRes = mapResults(results, ctx.req.nodes(), +// SnapshotCheckResponse::customHandlersResults); + Map>> cstRes = Collections.emptyMap(); checker.checkCustomHandlersResults(ctx.req.snapshotName(), cstRes); @@ -178,8 +180,13 @@ else if (ctx.req.allRestoreHandlers()) { Map errors0 = mapErrors(errors); if (!results.isEmpty()) { - Map> results0 = mapResults(results, ctx.req.nodes(), - SnapshotCheckResponse::partsHashes); +// Map> results0 = mapResults(results, ctx.req.nodes(), +// SnapshotCheckResponse::partsHashes); + Map> results0 = new HashMap<>(); + + results0.forEach((node, map) -> { + + }); IdleVerifyResultV2 chkRes = SnapshotChecker.reduceHashesResults(results0, errors0); @@ -201,7 +208,7 @@ private IgniteInternalFuture validateParts(SnapshotCheckP assert ctx != null; - if (ctx.locMeta == null) + if (F.isEmpty(ctx.locMetas)) return new GridFinishedFuture<>(); IgniteSnapshotManager snpMgr = kctx.cache().context().snapshotMgr(); @@ -210,7 +217,7 @@ private IgniteInternalFuture validateParts(SnapshotCheckP // Might be already finished by asynchronous leave of a required node. if (!phaseFut.isDone()) { - CompletableFuture workingFut; + CompletableFuture workingFut; if (req.incrementalIndex() > 0) { assert !req.allRestoreHandlers() : "Snapshot handlers aren't supported for incremental snapshot."; @@ -218,20 +225,56 @@ private IgniteInternalFuture validateParts(SnapshotCheckP workingFut = snpMgr.checker().checkIncrementalSnapshot(req.snapshotName(), req.snapshotPath(), req.incrementalIndex()); } else { - workingFut = req.allRestoreHandlers() - ? snpMgr.checker().invokeCustomHandlers(ctx.locMeta, req.snapshotPath(), req.groups(), true) - : snpMgr.checker().checkPartitions(ctx.locMeta, snpMgr.snapshotLocalDir(req.snapshotName(), req.snapshotPath()), - req.groups(), false, req.fullCheck(), false); + workingFut = new CompletableFuture<>(); + + Map> perMetaResults = new ConcurrentHashMap<>(ctx.locMetas.size(), 1.0f); + + for (SnapshotMetadata locMeta : ctx.locMetas) { + CompletableFuture metaFut; + + if (req.allRestoreHandlers()) + metaFut = snpMgr.checker().invokeCustomHandlers(locMeta, req.snapshotPath(), req.groups(), true); + else { + metaFut = snpMgr.checker().checkPartitions( + locMeta, + snpMgr.snapshotLocalDir(req.snapshotName(), req.snapshotPath()), + req.groups(), + false, + req.fullCheck(), + false + ); + } + + metaFut.whenComplete((res, err) -> { + if (err != null) + workingFut.completeExceptionally((Throwable)err); + else if (req.allRestoreHandlers()) { + Map> hndRes = (Map>)res; + + if (!F.isEmpty(hndRes)) + perMetaResults.put(F.first(hndRes.values()).node().consistentId(), hndRes); + } + else { + Map partRes = (Map)res; + + if (!F.isEmpty(partRes)) + perMetaResults.putIfAbsent(F.first(partRes.values()).consistentId(), partRes); + } + + if (perMetaResults.size() == ctx.locMetas.size()) + workingFut.complete(perMetaResults); + }); + } } workingFut.whenComplete((res, err) -> { if (err != null) - phaseFut.onDone(err); + phaseFut.onDone((Throwable)err); else { if (req.incrementalIndex() > 0) phaseFut.onDone(new SnapshotCheckResponse((IncrementalSnapshotCheckResult)res)); else - phaseFut.onDone(new SnapshotCheckResponse((Map)res)); + phaseFut.onDone(new SnapshotCheckResponse((Map)res)); } }); } @@ -358,9 +401,7 @@ private void reducePreparationAndMetasCheck( if (!metasCheck.isEmpty()) throw new IgniteSnapshotVerifyException(metasCheck); - List locMetas = metas.get(kctx.cluster().get().localNode()); - - ctx.locMeta = F.isEmpty(locMetas) ? null : locMetas.get(0); + ctx.locMetas = metas.get(kctx.cluster().get().localNode()); if (clusterOpFut != null) ctx.clusterMetas = metas; @@ -454,7 +495,7 @@ private static final class SnapshotCheckContext { private final GridFutureAdapter locProcFut = new GridFutureAdapter<>(); /** Local snapshot metadata. */ - @Nullable private SnapshotMetadata locMeta; + @Nullable private List locMetas; /** All the snapshot metadatas. */ @Nullable private Map> clusterMetas; @@ -486,7 +527,7 @@ private static final class SnapshotCheckResponse implements Serializable { * @see #partsHashes() * @see #customHandlersResults() */ - @Nullable private final Map partsResults; + @Nullable private final Map partsResults; /** @see #incrementalResult() */ @Nullable private final IncrementalSnapshotCheckResult incRes; @@ -499,7 +540,7 @@ private SnapshotCheckResponse(@Nullable List metas) { } /** Ctor for the phase 2 for normal snapshot. */ - private SnapshotCheckResponse(Map partsResults) { + private SnapshotCheckResponse(Map partsResults) { this.metas = null; this.partsResults = partsResults; this.incRes = null; @@ -521,8 +562,8 @@ private SnapshotCheckResponse(IncrementalSnapshotCheckResult incRes) { * Node's partition hashes for the phase 2. Is always {@code null} for the phase 1 or in case of incremental * snapshot. */ - private @Nullable Map partsHashes() { - return (Map)partsResults; + private @Nullable Map> partsHashes() { + return (Map>)partsResults; } /** @@ -531,8 +572,8 @@ private SnapshotCheckResponse(IncrementalSnapshotCheckResult incRes) { * * @see IgniteSnapshotManager#handlers() */ - private @Nullable Map> customHandlersResults() { - return (Map>)partsResults; + private @Nullable Map>> customHandlersResults() { + return (Map>>)partsResults; } /** Incremental snapshot result for the phase 2. Is always {@code null} for the phase 1 or in case of normal snapshot. */ From 230530c8bc1bab9db721802f00b2228d6154d56a Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Tue, 15 Oct 2024 01:34:20 +0300 Subject: [PATCH 09/27] impl --- .../cache/VerifyBackupPartitionsTaskV2.java | 8 +- .../snapshot/SnapshotCheckProcess.java | 98 ++++++++++++++----- .../persistence/snapshot/SnapshotChecker.java | 40 ++++---- 3 files changed, 99 insertions(+), 47 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java index 048b39020b7a5..83db900bdee32 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java @@ -146,7 +146,7 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter results) { Map ex = new HashMap<>(); - Map> hashes = new HashMap<>(); + Map>> hashes = new HashMap<>(); for (ComputeJobResult res : results) { if (res.getException() != null) { @@ -155,7 +155,11 @@ public static IdleVerifyResultV2 reduce0(List results) { continue; } - hashes.put(res.getNode(), res.getData()); + Map> nodePartsRes = hashes.computeIfAbsent(res.getNode(), node -> new HashMap<>()); + + Map nodeData = res.getData(); + + nodeData.forEach((partKey, partHash) -> nodePartsRes.computeIfAbsent(partKey, k -> new ArrayList<>()).add(partHash)); } return SnapshotChecker.reduceHashesResults(hashes, ex); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index ec270bc9d871a..64c1dab645171 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -18,8 +18,10 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; import java.io.Serializable; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -164,9 +166,20 @@ else if (ctx.req.allRestoreHandlers()) { if (!errors.isEmpty()) throw F.firstValue(errors); -// Map>> cstRes = mapResults(results, ctx.req.nodes(), -// SnapshotCheckResponse::customHandlersResults); - Map>> cstRes = Collections.emptyMap(); + // Check responses: node -> consistentId -> handler name -> handler result. + Map>>> cstRes = new HashMap<>(); + + results.forEach((nodeId, resp) -> { + if (resp != null) { + resp.customHandlersResults().forEach((consId, hndResMap) -> { + Map>> nodePerConsIdRes + = cstRes.computeIfAbsent(kctx.cluster().get().localNode(), n -> new HashMap<>()); + + hndResMap.forEach((hndId, hndRes) -> + nodePerConsIdRes.computeIfAbsent(consId, cstId -> new HashMap<>()).put(hndId, hndRes)); + }); + } + }); checker.checkCustomHandlersResults(ctx.req.snapshotName(), cstRes); @@ -180,12 +193,18 @@ else if (ctx.req.allRestoreHandlers()) { Map errors0 = mapErrors(errors); if (!results.isEmpty()) { -// Map> results0 = mapResults(results, ctx.req.nodes(), -// SnapshotCheckResponse::partsHashes); - Map> results0 = new HashMap<>(); + Map>> results0 = new HashMap<>(); - results0.forEach((node, map) -> { + results.forEach((nodeId, resp) -> { + if (resp != null) { + resp.partsHashes().forEach((consId, partsRes) -> { + Map> partsHashes + = results0.computeIfAbsent(kctx.cluster().get().localNode(), map -> new HashMap<>()); + partsRes.forEach((partKey, partHash) -> partsHashes.computeIfAbsent(partKey, k -> new ArrayList<>()) + .add(partHash)); + }); + } }); IdleVerifyResultV2 chkRes = SnapshotChecker.reduceHashesResults(results0, errors0); @@ -208,7 +227,7 @@ private IgniteInternalFuture validateParts(SnapshotCheckP assert ctx != null; - if (F.isEmpty(ctx.locMetas)) + if (F.isEmpty(ctx.metasToProcess)) return new GridFinishedFuture<>(); IgniteSnapshotManager snpMgr = kctx.cache().context().snapshotMgr(); @@ -227,10 +246,10 @@ private IgniteInternalFuture validateParts(SnapshotCheckP else { workingFut = new CompletableFuture<>(); - Map> perMetaResults = new ConcurrentHashMap<>(ctx.locMetas.size(), 1.0f); + Map> perMetaResults = new ConcurrentHashMap<>(ctx.metasToProcess.size(), 1.0f); - for (SnapshotMetadata locMeta : ctx.locMetas) { - CompletableFuture metaFut; + for (SnapshotMetadata locMeta : ctx.metasToProcess) { + CompletableFuture metaFut; if (req.allRestoreHandlers()) metaFut = snpMgr.checker().invokeCustomHandlers(locMeta, req.snapshotPath(), req.groups(), true); @@ -247,7 +266,7 @@ private IgniteInternalFuture validateParts(SnapshotCheckP metaFut.whenComplete((res, err) -> { if (err != null) - workingFut.completeExceptionally((Throwable)err); + workingFut.completeExceptionally(err); else if (req.allRestoreHandlers()) { Map> hndRes = (Map>)res; @@ -261,7 +280,7 @@ else if (req.allRestoreHandlers()) { perMetaResults.putIfAbsent(F.first(partRes.values()).consistentId(), partRes); } - if (perMetaResults.size() == ctx.locMetas.size()) + if (perMetaResults.size() == ctx.metasToProcess.size()) workingFut.complete(perMetaResults); }); } @@ -274,7 +293,7 @@ else if (req.allRestoreHandlers()) { if (req.incrementalIndex() > 0) phaseFut.onDone(new SnapshotCheckResponse((IncrementalSnapshotCheckResult)res)); else - phaseFut.onDone(new SnapshotCheckResponse((Map)res)); + phaseFut.onDone(new SnapshotCheckResponse((Map>)res)); } }); } @@ -388,7 +407,7 @@ private void reducePreparationAndMetasCheck( results.forEach((nodeId, nodeRes) -> { // A node might be not required. It gives null result. But a required node might have invalid empty result // which must be validated. - if (ctx.req.nodes().contains(nodeId) && baseline(nodeId)) { + if (ctx.req.nodes().contains(nodeId) && baseline(nodeId) && !F.isEmpty(nodeRes.metas)) { assert nodeRes != null && nodeRes.partsResults == null; metas.put(kctx.cluster().get().node(nodeId), nodeRes.metas); @@ -401,7 +420,8 @@ private void reducePreparationAndMetasCheck( if (!metasCheck.isEmpty()) throw new IgniteSnapshotVerifyException(metasCheck); - ctx.locMetas = metas.get(kctx.cluster().get().localNode()); + // If the topology is lesser that the snapshot's, we have to check partitions not only of current node. + ctx.metasToProcess = assingMetasToWork(metas); if (clusterOpFut != null) ctx.clusterMetas = metas; @@ -422,6 +442,32 @@ private void reducePreparationAndMetasCheck( } } + /** */ + private List assingMetasToWork(Map> clusterMetas) { + List locMetas = clusterMetas.get(kctx.cluster().get().localNode()); + + if (F.isEmpty(locMetas)) + return null; + + UUID minOrderDataNodeId = clusterMetas.keySet().stream().sorted(new Comparator<>() { + @Override public int compare(ClusterNode o1, ClusterNode o2) { + return Long.compare(o1.order(), o2.order()); + } + }).map(ClusterNode::id).findFirst().get(); + + if (minOrderDataNodeId.equals(kctx.localNodeId())) { + Collection onlineDataNodesIds = clusterMetas.keySet().stream().map(node -> node.consistentId().toString()) + .collect(Collectors.toSet()); + + locMetas.removeIf(meta -> !meta.consistentId().equals(kctx.cluster().get().localNode().consistentId()) + && onlineDataNodesIds.contains(meta.consistentId())); + } + else + locMetas = Collections.singletonList(F.first(locMetas)); + + return locMetas; + } + /** * Starts the snapshot validation process. * @@ -495,7 +541,7 @@ private static final class SnapshotCheckContext { private final GridFutureAdapter locProcFut = new GridFutureAdapter<>(); /** Local snapshot metadata. */ - @Nullable private List locMetas; + @Nullable private List metasToProcess; /** All the snapshot metadatas. */ @Nullable private Map> clusterMetas; @@ -524,10 +570,12 @@ private static final class SnapshotCheckResponse implements Serializable { @Nullable private final List metas; /** + * Partitions check result by node's consistent id. + * * @see #partsHashes() * @see #customHandlersResults() */ - @Nullable private final Map partsResults; + @Nullable private final Map> partsResults; /** @see #incrementalResult() */ @Nullable private final IncrementalSnapshotCheckResult incRes; @@ -540,7 +588,7 @@ private SnapshotCheckResponse(@Nullable List metas) { } /** Ctor for the phase 2 for normal snapshot. */ - private SnapshotCheckResponse(Map partsResults) { + private SnapshotCheckResponse(Map> partsResults) { this.metas = null; this.partsResults = partsResults; this.incRes = null; @@ -559,21 +607,21 @@ private SnapshotCheckResponse(IncrementalSnapshotCheckResult incRes) { } /** - * Node's partition hashes for the phase 2. Is always {@code null} for the phase 1 or in case of incremental - * snapshot. + * Node's partition hashes per consistent id for the phase 2. Is always {@code null} for the phase 1 or in case of + * incremental snapshot. */ private @Nullable Map> partsHashes() { return (Map>)partsResults; } /** - * Results of the custom handlers for the phase 2. Is always {@code null} for the phase 1 or in case of incremental - * snapshot. + * Results of the custom handlers per consistent id for the phase 2. Is always {@code null} for the phase 1 or in case of + * incremental snapshot. * * @see IgniteSnapshotManager#handlers() */ - private @Nullable Map>> customHandlersResults() { - return (Map>>)partsResults; + private @Nullable Map>> customHandlersResults() { + return (Map>>)partsResults; } /** Incremental snapshot result for the phase 2. Is always {@code null} for the phase 1 or in case of normal snapshot. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java index 994a362221f46..8bda8c83d4677 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java @@ -634,19 +634,16 @@ else if (!res.exceptions().isEmpty()) /** */ public static IdleVerifyResultV2 reduceHashesResults( - Map> results, + Map>> results, Map ex ) { Map> clusterHashes = new HashMap<>(); - results.forEach((node, nodeHashes) -> { + results.forEach((node, partsHashes) -> { assert ex.get(node) == null; - for (Map.Entry e : nodeHashes.entrySet()) { - List records = clusterHashes.computeIfAbsent(e.getKey(), k -> new ArrayList<>()); - - records.add(e.getValue()); - } + partsHashes.forEach((partKey, partsResults) -> clusterHashes.computeIfAbsent(partKey, + k -> new ArrayList<>()).addAll(partsResults)); }); if (results.size() != ex.size()) @@ -1024,32 +1021,35 @@ public CompletableFuture> checkPartit /** * Checks results of the internal and custon snapshot validation handlres. Throws exception if a validation error occurs. - * + * @param snpName Snapshot name + * @param results Results: checking node -> snapshot's node consistend id -> custom handler id -> handler result. * @see #invokeCustomHandlers(String, String, String, Collection, boolean) */ public void checkCustomHandlersResults( String snpName, - Map>> results + Map>>> results ) throws Exception { Map>> clusterResults = new HashMap<>(); Collection execNodes = new ArrayList<>(results.size()); - for (Map.Entry>> nodeRes : results.entrySet()) { - ClusterNode node = nodeRes.getKey(); + for (Map.Entry>>> nodeRes : results.entrySet()) { + for (Map.Entry>> nodeConsIdRes : nodeRes.getValue().entrySet()) { + ClusterNode node = nodeRes.getKey(); - // Depending on the job mapping, we can get several different results from one node. - execNodes.add(node.id()); + // Depending on the job mapping, we can get several different results from one node. + execNodes.add(node.id()); - assert nodeRes.getValue() != null : "At least the default snapshot restore handler should have been executed "; + assert nodeRes.getValue() != null : "At least the default snapshot restore handler should have been executed "; - for (Map.Entry> nodeHndRes : nodeRes.getValue().entrySet()) { - String hndName = nodeHndRes.getKey(); - SnapshotHandlerResult hndRes = nodeHndRes.getValue(); + for (Map.Entry> nodeHndRes : nodeConsIdRes.getValue().entrySet()) { + String hndName = nodeHndRes.getKey(); + SnapshotHandlerResult hndRes = nodeHndRes.getValue(); - if (hndRes.error() != null) - throw hndRes.error(); + if (hndRes.error() != null) + throw hndRes.error(); - clusterResults.computeIfAbsent(hndName, v -> new ArrayList<>()).add(hndRes); + clusterResults.computeIfAbsent(hndName, v -> new ArrayList<>()).add(hndRes); + } } } From ef34091dfb5cc249a86f31bb9f12cef0479ead4d Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Tue, 15 Oct 2024 17:02:25 +0300 Subject: [PATCH 10/27] fix --- .../cache/persistence/snapshot/SnapshotCheckProcess.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index 64c1dab645171..e2558755aad9d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -30,6 +30,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.ignite.IgniteException; @@ -248,6 +249,8 @@ private IgniteInternalFuture validateParts(SnapshotCheckP Map> perMetaResults = new ConcurrentHashMap<>(ctx.metasToProcess.size(), 1.0f); + AtomicInteger metasProcessed = new AtomicInteger(ctx.metasToProcess.size()); + for (SnapshotMetadata locMeta : ctx.metasToProcess) { CompletableFuture metaFut; @@ -280,7 +283,7 @@ else if (req.allRestoreHandlers()) { perMetaResults.putIfAbsent(F.first(partRes.values()).consistentId(), partRes); } - if (perMetaResults.size() == ctx.metasToProcess.size()) + if (metasProcessed.decrementAndGet() == 0) workingFut.complete(perMetaResults); }); } From 4e6d810221f0ce91ed48598b961838468bc64c42 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Tue, 15 Oct 2024 17:02:25 +0300 Subject: [PATCH 11/27] +test --- .../snapshot/SnapshotCheckProcess.java | 240 ++++++++++++------ .../persistence/snapshot/SnapshotChecker.java | 12 +- .../IgniteClusterSnapshotCheckTest.java | 52 ++++ 3 files changed, 217 insertions(+), 87 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index 64c1dab645171..8cbb800285650 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -30,6 +30,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.ignite.IgniteException; @@ -169,17 +170,23 @@ else if (ctx.req.allRestoreHandlers()) { // Check responses: node -> consistentId -> handler name -> handler result. Map>>> cstRes = new HashMap<>(); - results.forEach((nodeId, resp) -> { - if (resp != null) { - resp.customHandlersResults().forEach((consId, hndResMap) -> { - Map>> nodePerConsIdRes - = cstRes.computeIfAbsent(kctx.cluster().get().localNode(), n -> new HashMap<>()); + for (Map.Entry respE : results.entrySet()) { + SnapshotCheckResponse resp = respE.getValue(); - hndResMap.forEach((hndId, hndRes) -> - nodePerConsIdRes.computeIfAbsent(consId, cstId -> new HashMap<>()).put(hndId, hndRes)); - }); - } - }); + if (resp == null) + break; + + if (!F.isEmpty(resp.exceptions())) + throw F.firstValue(resp.exceptions()); + + resp.customHandlersResults().forEach((consId, hndResMap) -> { + Map>> nodePerConsIdRes + = cstRes.computeIfAbsent(kctx.cluster().get().localNode(), n -> new HashMap<>()); + + hndResMap.forEach((hndId, hndRes) -> + nodePerConsIdRes.computeIfAbsent(consId, cstId -> new HashMap<>()).put(hndId, hndRes)); + }); + } checker.checkCustomHandlersResults(ctx.req.snapshotName(), cstRes); @@ -195,17 +202,24 @@ else if (ctx.req.allRestoreHandlers()) { if (!results.isEmpty()) { Map>> results0 = new HashMap<>(); - results.forEach((nodeId, resp) -> { - if (resp != null) { - resp.partsHashes().forEach((consId, partsRes) -> { - Map> partsHashes - = results0.computeIfAbsent(kctx.cluster().get().localNode(), map -> new HashMap<>()); + for (Map.Entry respE : results.entrySet()) { + UUID nodeId = respE.getKey(); + SnapshotCheckResponse resp = respE.getValue(); - partsRes.forEach((partKey, partHash) -> partsHashes.computeIfAbsent(partKey, k -> new ArrayList<>()) - .add(partHash)); - }); - } - }); + if (resp == null) + break; + + if (!F.isEmpty(resp.exceptions())) + errors0.putIfAbsent(kctx.cluster().get().node(nodeId), asException(F.firstValue(resp.exceptions()))); + + resp.partsHashes().forEach((consId, partsRes) -> { + Map> partsHashes + = results0.computeIfAbsent(kctx.cluster().get().localNode(), map -> new HashMap<>()); + + partsRes.forEach((partKey, partHash) -> partsHashes.computeIfAbsent(partKey, k -> new ArrayList<>()) + .add(partHash)); + }); + } IdleVerifyResultV2 chkRes = SnapshotChecker.reduceHashesResults(results0, errors0); @@ -227,7 +241,7 @@ private IgniteInternalFuture validateParts(SnapshotCheckP assert ctx != null; - if (F.isEmpty(ctx.metasToProcess)) + if (F.isEmpty(ctx.metasToProc)) return new GridFinishedFuture<>(); IgniteSnapshotManager snpMgr = kctx.cache().context().snapshotMgr(); @@ -243,48 +257,8 @@ private IgniteInternalFuture validateParts(SnapshotCheckP workingFut = snpMgr.checker().checkIncrementalSnapshot(req.snapshotName(), req.snapshotPath(), req.incrementalIndex()); } - else { - workingFut = new CompletableFuture<>(); - - Map> perMetaResults = new ConcurrentHashMap<>(ctx.metasToProcess.size(), 1.0f); - - for (SnapshotMetadata locMeta : ctx.metasToProcess) { - CompletableFuture metaFut; - - if (req.allRestoreHandlers()) - metaFut = snpMgr.checker().invokeCustomHandlers(locMeta, req.snapshotPath(), req.groups(), true); - else { - metaFut = snpMgr.checker().checkPartitions( - locMeta, - snpMgr.snapshotLocalDir(req.snapshotName(), req.snapshotPath()), - req.groups(), - false, - req.fullCheck(), - false - ); - } - - metaFut.whenComplete((res, err) -> { - if (err != null) - workingFut.completeExceptionally(err); - else if (req.allRestoreHandlers()) { - Map> hndRes = (Map>)res; - - if (!F.isEmpty(hndRes)) - perMetaResults.put(F.first(hndRes.values()).node().consistentId(), hndRes); - } - else { - Map partRes = (Map)res; - - if (!F.isEmpty(partRes)) - perMetaResults.putIfAbsent(F.first(partRes.values()).consistentId(), partRes); - } - - if (perMetaResults.size() == ctx.metasToProcess.size()) - workingFut.complete(perMetaResults); - }); - } - } + else + workingFut = validatePartitionsFuture(ctx); workingFut.whenComplete((res, err) -> { if (err != null) @@ -293,7 +267,7 @@ else if (req.allRestoreHandlers()) { if (req.incrementalIndex() > 0) phaseFut.onDone(new SnapshotCheckResponse((IncrementalSnapshotCheckResult)res)); else - phaseFut.onDone(new SnapshotCheckResponse((Map>)res)); + phaseFut.onDone(new SnapshotCheckResponse((Map)res, req.incrementalIndex() > 0)); } }); } @@ -301,11 +275,69 @@ else if (req.allRestoreHandlers()) { return phaseFut; } + /** @return Composed partitions validating future regarding {@link SnapshotCheckContext#metasToProc}. */ + private CompletableFuture> validatePartitionsFuture(SnapshotCheckContext ctx) { + if (F.isEmpty(ctx.metasToProc)) + return CompletableFuture.completedFuture(null); + + // Per metas result: consistent id -> check result or an exception. + Map perMetaResults = new ConcurrentHashMap<>(ctx.metasToProc.size(), 1.0f); + + CompletableFuture> composedFut = new CompletableFuture<>(); + AtomicInteger metasProcessed = new AtomicInteger(ctx.metasToProc.size()); + + IgniteSnapshotManager snpMgr = kctx.cache().context().snapshotMgr(); + SnapshotCheckProcessRequest req = ctx.req; + + for (SnapshotMetadata locMeta : ctx.metasToProc) { + CompletableFuture metaFut; + + if (req.allRestoreHandlers()) + metaFut = snpMgr.checker().invokeCustomHandlers(locMeta, req.snapshotPath(), req.groups(), true); + else { + metaFut = snpMgr.checker().checkPartitions( + locMeta, + snpMgr.snapshotLocalDir(req.snapshotName(), req.snapshotPath()), + req.groups(), + false, + req.fullCheck(), + false + ); + } + + metaFut.whenComplete((res, err) -> { + if (err != null) + perMetaResults.put(locMeta.consistentId(), err); + else if (req.allRestoreHandlers()) { + Map> hndRes = (Map>)res; + + if (!F.isEmpty(hndRes)) + perMetaResults.put(F.first(hndRes.values()).node().consistentId().toString(), hndRes); + } + else { + Map partRes = (Map)res; + + if (!F.isEmpty(partRes)) + perMetaResults.putIfAbsent(F.first(partRes.values()).consistentId().toString(), partRes); + } + + if (metasProcessed.decrementAndGet() == 0) + composedFut.complete(perMetaResults); + }); + } + + return composedFut; + } + /** */ private Map mapErrors(Map errors) { return errors.entrySet().stream() - .collect(Collectors.toMap(e -> kctx.cluster().get().node(e.getKey()), - e -> e.getValue() instanceof Exception ? (Exception)e.getValue() : new IgniteException(e.getValue()))); + .collect(Collectors.toMap(e -> kctx.cluster().get().node(e.getKey()), e -> asException(e.getValue()))); + } + + /** */ + private static Exception asException(Throwable th) { + return th instanceof Exception ? (Exception)th : new IgniteException(th); } /** */ @@ -421,7 +453,7 @@ private void reducePreparationAndMetasCheck( throw new IgniteSnapshotVerifyException(metasCheck); // If the topology is lesser that the snapshot's, we have to check partitions not only of current node. - ctx.metasToProcess = assingMetasToWork(metas); + ctx.metasToProc = assingMetasToWork(metas); if (clusterOpFut != null) ctx.clusterMetas = metas; @@ -540,8 +572,11 @@ private static final class SnapshotCheckContext { /** Current process' future. Listens error, stop requests, etc. */ private final GridFutureAdapter locProcFut = new GridFutureAdapter<>(); - /** Local snapshot metadata. */ - @Nullable private List metasToProcess; + /** + * Metadatas to process on this node. Also indicates the snapshot part to validate on this node. + * @see #validatePartitionsFuture(SnapshotCheckContext) + */ + @Nullable private List metasToProc; /** All the snapshot metadatas. */ @Nullable private Map> clusterMetas; @@ -570,12 +605,14 @@ private static final class SnapshotCheckResponse implements Serializable { @Nullable private final List metas; /** - * Partitions check result by node's consistent id. - * * @see #partsHashes() * @see #customHandlersResults() + * @see #exceptions() */ - @Nullable private final Map> partsResults; + @Nullable private final Map> partsResults; + + /** @see #exceptions() */ + @Nullable private final Map exceptions; /** @see #incrementalResult() */ @Nullable private final IncrementalSnapshotCheckResult incRes; @@ -585,19 +622,59 @@ private SnapshotCheckResponse(@Nullable List metas) { this.metas = metas; this.partsResults = null; this.incRes = null; + this.exceptions = null; } - /** Ctor for the phase 2 for normal snapshot. */ - private SnapshotCheckResponse(Map> partsResults) { + /** + * Ctor for the phase 2 for normal snapshot. + * + * @param resultsPerConsId Partitions check result by node's consistent id as string: consistend id -> map of + * partition checks results by a partition id or an exception for current consistent id. + * @param allHandlers All handlers result flag. + */ + private SnapshotCheckResponse(Map resultsPerConsId, boolean allHandlers) { this.metas = null; - this.partsResults = partsResults; this.incRes = null; + + Map exceptions = new HashMap<>(); + + if (allHandlers) { + Map>> allHndResults = new HashMap<>(); + + resultsPerConsId.forEach((consId, consIdRes) -> { + assert consIdRes instanceof Throwable || consIdRes instanceof Map; + + if (consIdRes instanceof Throwable) + exceptions.put(consId, (Throwable)consIdRes); + else + allHndResults.put(consId, (Map>)consIdRes); + }); + + this.partsResults = allHndResults; + } + else { + Map> partsHashesResults = new HashMap<>(); + + resultsPerConsId.forEach((consId, consIdRes) -> { + assert consIdRes instanceof Throwable || consIdRes instanceof Map; + + if (consIdRes instanceof Throwable) + exceptions.put(consId, (Throwable)consIdRes); + else + partsHashesResults.put(consId, (Map)consIdRes); + }); + + this.partsResults = partsHashesResults; + } + + this.exceptions = exceptions; } /** Ctor for the phase 2 for incremental snapshot. */ private SnapshotCheckResponse(IncrementalSnapshotCheckResult incRes) { this.metas = null; this.partsResults = null; + this.exceptions = null; this.incRes = incRes; } @@ -606,12 +683,17 @@ private SnapshotCheckResponse(IncrementalSnapshotCheckResult incRes) { return metas; } + /** Exceptions found on phase 2 per consistent id. Is always {@code null} for the phase 1. */ + @Nullable private Map exceptions() { + return exceptions; + } + /** * Node's partition hashes per consistent id for the phase 2. Is always {@code null} for the phase 1 or in case of * incremental snapshot. */ - private @Nullable Map> partsHashes() { - return (Map>)partsResults; + private @Nullable Map> partsHashes() { + return (Map>)partsResults; } /** @@ -620,8 +702,8 @@ private SnapshotCheckResponse(IncrementalSnapshotCheckResult incRes) { * * @see IgniteSnapshotManager#handlers() */ - private @Nullable Map>> customHandlersResults() { - return (Map>>)partsResults; + private @Nullable Map>> customHandlersResults() { + return (Map>>)partsResults; } /** Incremental snapshot result for the phase 2. Is always {@code null} for the phase 1 or in case of normal snapshot. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java index 8bda8c83d4677..a1290ef782f72 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java @@ -637,17 +637,13 @@ public static IdleVerifyResultV2 reduceHashesResults( Map>> results, Map ex ) { - Map> clusterHashes = new HashMap<>(); + Map> hashesRes = new HashMap<>(); - results.forEach((node, partsHashes) -> { - assert ex.get(node) == null; - - partsHashes.forEach((partKey, partsResults) -> clusterHashes.computeIfAbsent(partKey, - k -> new ArrayList<>()).addAll(partsResults)); - }); + results.forEach((node, partsHashes) -> partsHashes.forEach((partKey, partHashLst) -> hashesRes.computeIfAbsent(partKey, + k -> new ArrayList<>()).addAll(partHashLst))); if (results.size() != ex.size()) - return new IdleVerifyResultV2(clusterHashes, ex); + return new IdleVerifyResultV2(hashesRes, ex); else return new IdleVerifyResultV2(ex); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java index 0e6fbf134c170..ac564bf624436 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java @@ -564,6 +564,58 @@ public void testClusterSnapshotCheckWithTwoCachesCheckNotCorrupted() throws Exce assertNotContains(log, b.toString(), "Failed to read page (CRC validation failed)"); } + /** */ + @Test + public void testCheckFromLesserTopology() throws Exception { + // {@link #corruptPartitionFile} affetchs an encrypted partition wrongly. + assumeFalse(encryption); + + int srvCnt = 3; + IdleVerifyResultV2 chkRes; + + IgniteEx client = startGridsWithSnapshot(srvCnt, CACHE_KEYS_RANGE, true, true); + + for (int i = 1; i <= srvCnt; ++i) { + int i0 = i; + + chkRes = snp(client).checkSnapshot(SNAPSHOT_NAME, null).get(getTestTimeout()).idleVerifyResult(); + assertTrue(chkRes.exceptions().isEmpty()); + assertFalse(chkRes.hasConflicts()); + + if (i == srvCnt) + break; + + stopGrid(i); + assertTrue(waitForCondition(() -> client.cluster().forServers().nodes().size() == srvCnt - i0, getTestTimeout())); + } + + for (int i = 1; i < srvCnt; ++i) + startGrid(i); + + assertTrue(waitForCondition(() -> client.cluster().forServers().nodes().size() == srvCnt, getTestTimeout())); + + // Now ensure that a bad partition is detected. + corruptPartitionFile(grid(1), SNAPSHOT_NAME, dfltCacheCfg, 3); + + stopGrid(1); + assertTrue(waitForCondition(() -> client.cluster().forServers().nodes().size() == 2, getTestTimeout())); + + for (int i = 2; i <= srvCnt; ++i) { + int i0 = i; + + chkRes = snp(client).checkSnapshot(SNAPSHOT_NAME, null).get(getTestTimeout()).idleVerifyResult(); + + assertFalse(chkRes.exceptions().isEmpty()); + assertTrue(X.hasCause(F.first(chkRes.exceptions().values()), IgniteDataIntegrityViolationException.class)); + + if (i == srvCnt) + break; + + stopGrid(i); + assertTrue(waitForCondition(() -> client.cluster().forServers().nodes().size() == srvCnt - i0, getTestTimeout())); + } + } + /** @throws Exception If fails. */ @Test public void testClusterSnapshotCheckWithTwoCachesCheckTwoCaches() throws Exception { From d0b202f3beaf8a7c643d7352248ae243ce5e0dc8 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 16 Oct 2024 00:29:46 +0300 Subject: [PATCH 12/27] + refactor inc --- .../snapshot/SnapshotCheckProcess.java | 515 ++++++++++-------- .../persistence/snapshot/SnapshotChecker.java | 6 +- .../IgniteClusterSnapshotCheckTest.java | 14 +- 3 files changed, 295 insertions(+), 240 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index 8cbb800285650..da083b171ab57 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -31,7 +31,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; import java.util.stream.Collectors; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -72,10 +71,10 @@ public class SnapshotCheckProcess { private final Map> clusterOpFuts = new ConcurrentHashMap<>(); /** Check metas first phase subprocess. */ - private final DistributedProcess phase1CheckMetas; + private final DistributedProcess phase1CheckMetas; /** Partition hashes second phase subprocess. */ - private final DistributedProcess phase2PartsHashes; + private final DistributedProcess phase2PartsHashes; /** Stop node lock. */ private boolean nodeStopping; @@ -132,7 +131,7 @@ void interrupt(Throwable err) { /** Phase 2 and process finish. */ private IgniteInternalFuture reduceValidatePartsAndFinish( UUID reqId, - Map results, + Map results, Map errors ) { SnapshotCheckContext ctx = context(null, reqId); @@ -150,90 +149,133 @@ private IgniteInternalFuture reduceValidatePartsAndFinish( if (clusterOpFut == null) return new GridFinishedFuture<>(); - assert results.values().stream().noneMatch(res -> res != null && res.metas != null); + if (ctx.req.incrementalIndex() > 0) + reduceIncrementalCheckResults(ctx.req.nodes(), ctx.clusterMetas, results, errors, clusterOpFut); + else if (ctx.req.allRestoreHandlers()) + reduceCustomHandlersResults(ctx, results, errors, clusterOpFut); + else + reducePartitionsHashesResults(ctx.clusterMetas, results, errors, clusterOpFut); + + return new GridFinishedFuture<>(); + } + /** */ + private void reduceIncrementalCheckResults( + Set requiredNodes, + Map> clusterMetas, + Map results, + Map errors, + GridFutureAdapter fut + ) { SnapshotChecker checker = kctx.cache().context().snapshotMgr().checker(); - if (ctx.req.incrementalIndex() > 0) { - IdleVerifyResultV2 chkRes = checker.reduceIncrementalResults( - mapResults(results, ctx.req.nodes(), SnapshotCheckResponse::incrementalResult), - mapErrors(errors) - ); + Map> reduced = new HashMap<>(); - clusterOpFut.onDone(new SnapshotPartitionsVerifyResult(ctx.clusterMetas, chkRes)); + for (Map.Entry resE : results.entrySet()) { + UUID nodeId = resE.getKey(); + SnapshotCheckIncrementalResponse incResp = (SnapshotCheckIncrementalResponse)resE.getValue(); + + if (requiredNodes.contains(nodeId) || incResp == null) + break; + + ClusterNode node = kctx.cluster().get().node(nodeId); + + incResp.incremenlatSnapshotResults().forEach((consId, incResPerConsId) -> + reduced.computeIfAbsent(node, nid -> new ArrayList<>()).add(incResPerConsId)); + + if (F.isEmpty(incResp.exceptions())) + break; + + errors.putIfAbsent(nodeId, asException(F.firstValue(incResp.exceptions()))); } - else if (ctx.req.allRestoreHandlers()) { - try { - if (!errors.isEmpty()) - throw F.firstValue(errors); - // Check responses: node -> consistentId -> handler name -> handler result. - Map>>> cstRes = new HashMap<>(); + IdleVerifyResultV2 chkRes = checker.reduceIncrementalResults(reduced, mapErrors(errors)); - for (Map.Entry respE : results.entrySet()) { - SnapshotCheckResponse resp = respE.getValue(); + fut.onDone(new SnapshotPartitionsVerifyResult(clusterMetas, chkRes)); + } - if (resp == null) - break; + /** */ + private void reduceCustomHandlersResults( + SnapshotCheckContext ctx, + Map results, + Map errors, + GridFutureAdapter fut + ) { + try { + if (!errors.isEmpty()) + throw F.firstValue(errors); - if (!F.isEmpty(resp.exceptions())) - throw F.firstValue(resp.exceptions()); + // Check responses: node -> consistentId -> handler name -> handler result. + Map>>> cstRes = new HashMap<>(); - resp.customHandlersResults().forEach((consId, hndResMap) -> { - Map>> nodePerConsIdRes - = cstRes.computeIfAbsent(kctx.cluster().get().localNode(), n -> new HashMap<>()); + for (Map.Entry respE : results.entrySet()) { + SnapshotCheckCustomHandlersResponse resp = (SnapshotCheckCustomHandlersResponse)respE.getValue(); - hndResMap.forEach((hndId, hndRes) -> - nodePerConsIdRes.computeIfAbsent(consId, cstId -> new HashMap<>()).put(hndId, hndRes)); - }); - } + if (resp == null) + break; - checker.checkCustomHandlersResults(ctx.req.snapshotName(), cstRes); + if (!F.isEmpty(resp.exceptions())) + throw F.firstValue(resp.exceptions()); - clusterOpFut.onDone(new SnapshotPartitionsVerifyResult(ctx.clusterMetas, null)); - } - catch (Throwable err) { - clusterOpFut.onDone(err); + resp.customHandlersResults().forEach((consId, hndResMap) -> { + Map>> nodePerConsIdRes + = cstRes.computeIfAbsent(kctx.cluster().get().localNode(), n -> new HashMap<>()); + + hndResMap.forEach((hndId, hndRes) -> + nodePerConsIdRes.computeIfAbsent(consId, cstId -> new HashMap<>()).put(hndId, hndRes)); + }); } - } - else { - Map errors0 = mapErrors(errors); - if (!results.isEmpty()) { - Map>> results0 = new HashMap<>(); + kctx.cache().context().snapshotMgr().checker().checkCustomHandlersResults(ctx.req.snapshotName(), cstRes); - for (Map.Entry respE : results.entrySet()) { - UUID nodeId = respE.getKey(); - SnapshotCheckResponse resp = respE.getValue(); + fut.onDone(new SnapshotPartitionsVerifyResult(ctx.clusterMetas, null)); + } + catch (Throwable err) { + fut.onDone(err); + } + } - if (resp == null) - break; + /** */ + private void reducePartitionsHashesResults( + Map> clusterMetas, + Map results, + Map errors, + GridFutureAdapter fut + ) { + Map errors0 = mapErrors(errors); - if (!F.isEmpty(resp.exceptions())) - errors0.putIfAbsent(kctx.cluster().get().node(nodeId), asException(F.firstValue(resp.exceptions()))); + if (!results.isEmpty()) { + Map>> results0 = new HashMap<>(); - resp.partsHashes().forEach((consId, partsRes) -> { - Map> partsHashes - = results0.computeIfAbsent(kctx.cluster().get().localNode(), map -> new HashMap<>()); + for (Map.Entry respE : results.entrySet()) { + UUID nodeId = respE.getKey(); + SnapshotCheckPartitionsHashesResponse resp = (SnapshotCheckPartitionsHashesResponse)respE.getValue(); - partsRes.forEach((partKey, partHash) -> partsHashes.computeIfAbsent(partKey, k -> new ArrayList<>()) - .add(partHash)); - }); - } + if (resp == null) + break; + + if (!F.isEmpty(resp.exceptions())) + errors0.putIfAbsent(kctx.cluster().get().node(nodeId), asException(F.firstValue(resp.exceptions()))); - IdleVerifyResultV2 chkRes = SnapshotChecker.reduceHashesResults(results0, errors0); + resp.partitionsHashes().forEach((consId, partsRes) -> { + Map> partsHashes + = results0.computeIfAbsent(kctx.cluster().get().localNode(), map -> new HashMap<>()); - clusterOpFut.onDone(new SnapshotPartitionsVerifyResult(ctx.clusterMetas, chkRes)); + partsRes.forEach((partKey, partHash) -> partsHashes.computeIfAbsent(partKey, k -> new ArrayList<>()) + .add(partHash)); + }); } - else - clusterOpFut.onDone(new IgniteSnapshotVerifyException(errors0)); - } - return new GridFinishedFuture<>(); + IdleVerifyResultV2 chkRes = SnapshotChecker.reduceHashesResults(results0, errors0); + + fut.onDone(new SnapshotPartitionsVerifyResult(clusterMetas, chkRes)); + } + else + fut.onDone(new IgniteSnapshotVerifyException(errors0)); } /** Phase 2 beginning. */ - private IgniteInternalFuture validateParts(SnapshotCheckProcessRequest req) { + private IgniteInternalFuture validateParts(SnapshotCheckProcessRequest req) { if (!req.nodes().contains(kctx.localNodeId())) return new GridFinishedFuture<>(); @@ -241,88 +283,129 @@ private IgniteInternalFuture validateParts(SnapshotCheckP assert ctx != null; - if (F.isEmpty(ctx.metasToProc)) + if (F.isEmpty(ctx.metas)) return new GridFinishedFuture<>(); - IgniteSnapshotManager snpMgr = kctx.cache().context().snapshotMgr(); - - GridFutureAdapter phaseFut = ctx.phaseFuture(); + GridFutureAdapter phaseFut = ctx.phaseFuture(); // Might be already finished by asynchronous leave of a required node. if (!phaseFut.isDone()) { - CompletableFuture workingFut; + CompletableFuture workingFut; if (req.incrementalIndex() > 0) { assert !req.allRestoreHandlers() : "Snapshot handlers aren't supported for incremental snapshot."; - workingFut = snpMgr.checker().checkIncrementalSnapshot(req.snapshotName(), req.snapshotPath(), req.incrementalIndex()); + workingFut = incrementalFuture(ctx); } + else if (req.allRestoreHandlers()) + workingFut = allHandlersFuture(ctx); else - workingFut = validatePartitionsFuture(ctx); + workingFut = partitionsHashesFuture(ctx); workingFut.whenComplete((res, err) -> { if (err != null) - phaseFut.onDone((Throwable)err); - else { - if (req.incrementalIndex() > 0) - phaseFut.onDone(new SnapshotCheckResponse((IncrementalSnapshotCheckResult)res)); - else - phaseFut.onDone(new SnapshotCheckResponse((Map)res, req.incrementalIndex() > 0)); - } + phaseFut.onDone(err); + else + phaseFut.onDone(res); }); } return phaseFut; } - /** @return Composed partitions validating future regarding {@link SnapshotCheckContext#metasToProc}. */ - private CompletableFuture> validatePartitionsFuture(SnapshotCheckContext ctx) { - if (F.isEmpty(ctx.metasToProc)) - return CompletableFuture.completedFuture(null); + /** @return A composed future of increment checks for each meta/consistent id regarding {@link SnapshotCheckContext#metas}. */ + private CompletableFuture incrementalFuture(SnapshotCheckContext ctx) { + assert !F.isEmpty(ctx.metas); + + // Per metas result: consistent id -> check result. + Map perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); + // Exceptions: consistent id -> exceptions. + Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); + AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); + CompletableFuture composedFut = new CompletableFuture<>(); + + for (SnapshotMetadata meta : ctx.metas) { + CompletableFuture workingFut = kctx.cache().context().snapshotMgr().checker() + .checkIncrementalSnapshot(ctx.req.snapshotName(), + ctx.req.snapshotPath(), ctx.req.incrementalIndex()); + + workingFut.whenComplete((res, err) -> { + if (err != null) + exceptions.put(meta.consistentId(), err); + else + perMetaResults.put(meta.consistentId(), res); + + if (metasProcessed.decrementAndGet() == 0) + composedFut.complete( new SnapshotCheckIncrementalResponse(perMetaResults, exceptions)); + }); + } - // Per metas result: consistent id -> check result or an exception. - Map perMetaResults = new ConcurrentHashMap<>(ctx.metasToProc.size(), 1.0f); + return composedFut; + } - CompletableFuture> composedFut = new CompletableFuture<>(); - AtomicInteger metasProcessed = new AtomicInteger(ctx.metasToProc.size()); + /** @return A composed future of partitions checks for each meta/consistent id regarding {@link SnapshotCheckContext#metas}. */ + private CompletableFuture partitionsHashesFuture(SnapshotCheckContext ctx) { + assert !F.isEmpty(ctx.metas); + + // Per metas result: consistent id -> check results per partition key. + Map> perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); + // Exceptions: consistent id -> exceptions. + Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); + CompletableFuture composedFut = new CompletableFuture<>(); + AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); IgniteSnapshotManager snpMgr = kctx.cache().context().snapshotMgr(); - SnapshotCheckProcessRequest req = ctx.req; - - for (SnapshotMetadata locMeta : ctx.metasToProc) { - CompletableFuture metaFut; - - if (req.allRestoreHandlers()) - metaFut = snpMgr.checker().invokeCustomHandlers(locMeta, req.snapshotPath(), req.groups(), true); - else { - metaFut = snpMgr.checker().checkPartitions( - locMeta, - snpMgr.snapshotLocalDir(req.snapshotName(), req.snapshotPath()), - req.groups(), - false, - req.fullCheck(), - false - ); - } + + for (SnapshotMetadata meta : ctx.metas) { + CompletableFuture> metaFut = snpMgr.checker().checkPartitions( + meta, + snpMgr.snapshotLocalDir(ctx.req.snapshotName(), ctx.req.snapshotPath()), + ctx.req.groups(), + false, + ctx.req.fullCheck(), + false + ); metaFut.whenComplete((res, err) -> { if (err != null) - perMetaResults.put(locMeta.consistentId(), err); - else if (req.allRestoreHandlers()) { - Map> hndRes = (Map>)res; + exceptions.put(meta.consistentId(), err); + else if (!F.isEmpty(res)) + perMetaResults.put(meta.consistentId(), res); - if (!F.isEmpty(hndRes)) - perMetaResults.put(F.first(hndRes.values()).node().consistentId().toString(), hndRes); - } - else { - Map partRes = (Map)res; + if (metasProcessed.decrementAndGet() == 0) + composedFut.complete(new SnapshotCheckPartitionsHashesResponse(perMetaResults, exceptions)); + }); + } - if (!F.isEmpty(partRes)) - perMetaResults.putIfAbsent(F.first(partRes.values()).consistentId().toString(), partRes); - } + return composedFut; + } + + /** + * @return A composed future of all the snapshot handlers for each meta/consistent id regarding {@link SnapshotCheckContext#metas}. + * @see IgniteSnapshotManager#handlers() + */ + private CompletableFuture allHandlersFuture(SnapshotCheckContext ctx) { + assert !F.isEmpty(ctx.metas); + + // Per metas result: consistent id -> check result per handler id. + Map>> perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); + // Exceptions: consistent id -> exceptions. + Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); + CompletableFuture composedFut = new CompletableFuture<>(); + AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); + + for (SnapshotMetadata meta : ctx.metas) { + CompletableFuture>> metaFut = kctx.cache().context().snapshotMgr().checker() + .invokeCustomHandlers(meta, ctx.req.snapshotPath(), ctx.req.groups(), true); + + metaFut.whenComplete((res, err) -> { + if (err != null) + exceptions.put(meta.consistentId(), err); + else if (!F.isEmpty(res)) + perMetaResults.put(meta.consistentId(), res); if (metasProcessed.decrementAndGet() == 0) - composedFut.complete(perMetaResults); + composedFut.complete(new SnapshotCheckCustomHandlersResponse(perMetaResults, exceptions)); }); } @@ -340,17 +423,6 @@ private static Exception asException(Throwable th) { return th instanceof Exception ? (Exception)th : new IgniteException(th); } - /** */ - private Map mapResults( - Map results, - Set requiredNodes, - Function resExtractor - ) { - return results.entrySet().stream() - .filter(e -> requiredNodes.contains(e.getKey()) && e.getValue() != null) - .collect(Collectors.toMap(e -> kctx.cluster().get().node(e.getKey()), e -> resExtractor.apply(e.getValue()))); - } - /** * @param snpName Snapshot name. If {@code null}, ignored. * @param reqId If {@code ctxId} is {@code null}, is used to find the operation context. @@ -363,7 +435,7 @@ private Map mapResults( } /** Phase 1 beginning: prepare, collect and check local metas. */ - private IgniteInternalFuture prepareAndCheckMetas(SnapshotCheckProcessRequest req) { + private IgniteInternalFuture prepareAndCheckMetas(SnapshotCheckProcessRequest req) { if (!req.nodes().contains(kctx.localNodeId())) return new GridFinishedFuture<>(); @@ -390,7 +462,7 @@ private IgniteInternalFuture prepareAndCheckMetas(Snapsho Collection grpIds = F.isEmpty(req.groups()) ? null : F.viewReadOnly(req.groups(), CU::cacheId); - GridFutureAdapter phaseFut = ctx.phaseFuture(); + GridFutureAdapter phaseFut = ctx.phaseFuture(); // Might be already finished by asynchronous leave of a required node. if (!phaseFut.isDone()) { @@ -403,7 +475,7 @@ private IgniteInternalFuture prepareAndCheckMetas(Snapsho if (err != null) phaseFut.onDone(err); else - phaseFut.onDone(new SnapshotCheckResponse(locMetas)); + phaseFut.onDone(new SnapshotCheckMetasResponse(locMetas)); }); } @@ -413,7 +485,7 @@ private IgniteInternalFuture prepareAndCheckMetas(Snapsho /** Phase 1 end. */ private void reducePreparationAndMetasCheck( UUID reqId, - Map results, + Map results, Map errors ) { SnapshotCheckContext ctx = context(null, reqId); @@ -439,10 +511,10 @@ private void reducePreparationAndMetasCheck( results.forEach((nodeId, nodeRes) -> { // A node might be not required. It gives null result. But a required node might have invalid empty result // which must be validated. - if (ctx.req.nodes().contains(nodeId) && baseline(nodeId) && !F.isEmpty(nodeRes.metas)) { - assert nodeRes != null && nodeRes.partsResults == null; + if (ctx.req.nodes().contains(nodeId) && baseline(nodeId) && !F.isEmpty(nodeRes.metas())) { + assert nodeRes != null; - metas.put(kctx.cluster().get().node(nodeId), nodeRes.metas); + metas.put(kctx.cluster().get().node(nodeId), nodeRes.metas()); } }); @@ -453,7 +525,7 @@ private void reducePreparationAndMetasCheck( throw new IgniteSnapshotVerifyException(metasCheck); // If the topology is lesser that the snapshot's, we have to check partitions not only of current node. - ctx.metasToProc = assingMetasToWork(metas); + ctx.metas = assingMetasToWork(metas); if (clusterOpFut != null) ctx.clusterMetas = metas; @@ -570,13 +642,13 @@ private static final class SnapshotCheckContext { private final SnapshotCheckProcessRequest req; /** Current process' future. Listens error, stop requests, etc. */ - private final GridFutureAdapter locProcFut = new GridFutureAdapter<>(); + private final GridFutureAdapter locProcFut = new GridFutureAdapter<>(); /** - * Metadatas to process on this node. Also indicates the snapshot part to validate on this node. - * @see #validatePartitionsFuture(SnapshotCheckContext) + * Metadatas to process on this node. Also indicates the snapshot parts to check on this node. + * @see #partitionsHashesFuture(SnapshotCheckContext) */ - @Nullable private List metasToProc; + @Nullable private List metas; /** All the snapshot metadatas. */ @Nullable private Map> clusterMetas; @@ -596,119 +668,112 @@ private GridFutureAdapter phaseFuture() { } } - /** A DTO used to transfer nodes' results for the both phases. */ - private static final class SnapshotCheckResponse implements Serializable { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** @see #metas() */ - @Nullable private final List metas; + /** A DTO base to transfer nodes' results for the both phases. */ + private abstract static class AbstractSnapshotCheckResponse implements Serializable { + /** The result. */ + protected Object result; - /** - * @see #partsHashes() - * @see #customHandlersResults() - * @see #exceptions() - */ - @Nullable private final Map> partsResults; + /** Exceptions per consistent id. */ + @Nullable private Map exceptions; - /** @see #exceptions() */ - @Nullable private final Map exceptions; + /** */ + private AbstractSnapshotCheckResponse(Object result, @Nullable Map exceptions) { + assert result instanceof Serializable : "Snapshot check result is not serializable."; + assert exceptions == null || exceptions instanceof Serializable : "Snapshot check exceptions aren't serializable."; - /** @see #incrementalResult() */ - @Nullable private final IncrementalSnapshotCheckResult incRes; - - /** Ctor for the phase 1. */ - private SnapshotCheckResponse(@Nullable List metas) { - this.metas = metas; - this.partsResults = null; - this.incRes = null; - this.exceptions = null; + this.result = result; + this.exceptions = exceptions == null ? null : Collections.unmodifiableMap(exceptions); } - /** - * Ctor for the phase 2 for normal snapshot. - * - * @param resultsPerConsId Partitions check result by node's consistent id as string: consistend id -> map of - * partition checks results by a partition id or an exception for current consistent id. - * @param allHandlers All handlers result flag. - */ - private SnapshotCheckResponse(Map resultsPerConsId, boolean allHandlers) { - this.metas = null; - this.incRes = null; - - Map exceptions = new HashMap<>(); - - if (allHandlers) { - Map>> allHndResults = new HashMap<>(); - - resultsPerConsId.forEach((consId, consIdRes) -> { - assert consIdRes instanceof Throwable || consIdRes instanceof Map; - - if (consIdRes instanceof Throwable) - exceptions.put(consId, (Throwable)consIdRes); - else - allHndResults.put(consId, (Map>)consIdRes); - }); - - this.partsResults = allHndResults; - } - else { - Map> partsHashesResults = new HashMap<>(); - - resultsPerConsId.forEach((consId, consIdRes) -> { - assert consIdRes instanceof Throwable || consIdRes instanceof Map; - - if (consIdRes instanceof Throwable) - exceptions.put(consId, (Throwable)consIdRes); - else - partsHashesResults.put(consId, (Map)consIdRes); - }); + /** @return Exceptions per consistent id. */ + protected @Nullable Map exceptions() { + return exceptions; + } + } - this.partsResults = partsHashesResults; - } + /** A DTO to transfer metas result for phase 1. */ + private static final class SnapshotCheckMetasResponse extends AbstractSnapshotCheckResponse { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; - this.exceptions = exceptions; + /** */ + private SnapshotCheckMetasResponse(List result) { + super(result, null); } - /** Ctor for the phase 2 for incremental snapshot. */ - private SnapshotCheckResponse(IncrementalSnapshotCheckResult incRes) { - this.metas = null; - this.partsResults = null; - this.exceptions = null; - this.incRes = incRes; + /** @return All the snapshot metadatatas found on this node. */ + private List metas() { + return (List)result; } + } - /** Metas for the phase 1. Is always {@code null} for the phase 2. */ - @Nullable private List metas() { - return metas; + /** A DTO used to transfer partition hashes result for phase 2. */ + private static final class SnapshotCheckPartitionsHashesResponse extends AbstractSnapshotCheckResponse { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** + * @param result Partitions results per consistent id. + * @param exceptions Exceptions per consistent id. + */ + private SnapshotCheckPartitionsHashesResponse( + Map> result, + Map exceptions + ) { + super(result, exceptions); } - /** Exceptions found on phase 2 per consistent id. Is always {@code null} for the phase 1. */ - @Nullable private Map exceptions() { - return exceptions; + /** @return Partitions hashes per consistent id. */ + private Map> partitionsHashes() { + return (Map>)result; } + } + + /** + * A DTO used to transfer all the handlers results for phase 2. + * + * @see IgniteSnapshotManager#handlers(). + */ + private static final class SnapshotCheckCustomHandlersResponse extends AbstractSnapshotCheckResponse { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; /** - * Node's partition hashes per consistent id for the phase 2. Is always {@code null} for the phase 1 or in case of - * incremental snapshot. + * @param result Handlers results per consistent id. + * @param exceptions Exceptions per consistent id. */ - private @Nullable Map> partsHashes() { - return (Map>)partsResults; + private SnapshotCheckCustomHandlersResponse( + Map>> result, + Map exceptions + ) { + super(result, exceptions); + } + + /** @return Handlers results per consistent id. */ + private Map>> customHandlersResults() { + return (Map>>)result; } + } + + /** A DTO used to transfer incremental snapshot check results for phase 2. */ + private static final class SnapshotCheckIncrementalResponse extends AbstractSnapshotCheckResponse { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; /** - * Results of the custom handlers per consistent id for the phase 2. Is always {@code null} for the phase 1 or in case of - * incremental snapshot. - * - * @see IgniteSnapshotManager#handlers() + * @param result Incremental snapshot check results per consistent id. + * @param exceptions Exceptions per consistent id. */ - private @Nullable Map>> customHandlersResults() { - return (Map>>)partsResults; + private SnapshotCheckIncrementalResponse( + Map result, + Map exceptions + ) { + super(result, exceptions); } - /** Incremental snapshot result for the phase 2. Is always {@code null} for the phase 1 or in case of normal snapshot. */ - private @Nullable IncrementalSnapshotCheckResult incrementalResult() { - return incRes; + /** @return Incremental snapshot check results per consistent id. */ + private Map incremenlatSnapshotResults() { + return (Map)result; } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java index a1290ef782f72..5183fc7589b26 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java @@ -575,7 +575,7 @@ private Map readTxCachesData(File snpDir) throws Ignit /** */ public IdleVerifyResultV2 reduceIncrementalResults( - Map results, + Map> results, Map operationErrors ) { if (!operationErrors.isEmpty()) @@ -587,7 +587,7 @@ public IdleVerifyResultV2 reduceIncrementalResults( Map> partiallyCommittedTxs = new HashMap<>(); Map errors = new HashMap<>(); - results.forEach((node, res) -> { + results.forEach((node, resLst) -> resLst.forEach(res -> { if (res.exceptions().isEmpty() && errors.isEmpty()) { if (!F.isEmpty(res.partiallyCommittedTxs())) partiallyCommittedTxs.put(node, res.partiallyCommittedTxs()); @@ -620,7 +620,7 @@ public IdleVerifyResultV2 reduceIncrementalResults( } else if (!res.exceptions().isEmpty()) errors.put(node, F.first(res.exceptions())); - }); + })); // Add all missed pairs to conflicts. nodeTxHashMap.values().stream() diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java index ac564bf624436..1cdb4d23adc7f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java @@ -576,9 +576,8 @@ public void testCheckFromLesserTopology() throws Exception { IgniteEx client = startGridsWithSnapshot(srvCnt, CACHE_KEYS_RANGE, true, true); for (int i = 1; i <= srvCnt; ++i) { - int i0 = i; - chkRes = snp(client).checkSnapshot(SNAPSHOT_NAME, null).get(getTestTimeout()).idleVerifyResult(); + assertTrue(chkRes.exceptions().isEmpty()); assertFalse(chkRes.hasConflicts()); @@ -586,23 +585,15 @@ public void testCheckFromLesserTopology() throws Exception { break; stopGrid(i); - assertTrue(waitForCondition(() -> client.cluster().forServers().nodes().size() == srvCnt - i0, getTestTimeout())); } for (int i = 1; i < srvCnt; ++i) startGrid(i); - assertTrue(waitForCondition(() -> client.cluster().forServers().nodes().size() == srvCnt, getTestTimeout())); - // Now ensure that a bad partition is detected. corruptPartitionFile(grid(1), SNAPSHOT_NAME, dfltCacheCfg, 3); - stopGrid(1); - assertTrue(waitForCondition(() -> client.cluster().forServers().nodes().size() == 2, getTestTimeout())); - - for (int i = 2; i <= srvCnt; ++i) { - int i0 = i; - + for (int i = 1; i <= srvCnt; ++i) { chkRes = snp(client).checkSnapshot(SNAPSHOT_NAME, null).get(getTestTimeout()).idleVerifyResult(); assertFalse(chkRes.exceptions().isEmpty()); @@ -612,7 +603,6 @@ public void testCheckFromLesserTopology() throws Exception { break; stopGrid(i); - assertTrue(waitForCondition(() -> client.cluster().forServers().nodes().size() == srvCnt - i0, getTestTimeout())); } } From 060f0b181620d05b117d5ade23c5d0e27d2d5868 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 16 Oct 2024 02:35:43 +0300 Subject: [PATCH 13/27] + inc test --- ...ementalSnapshotCheckBeforeRestoreTest.java | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java index 8723234381629..fbd93f634b1ea 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java @@ -89,6 +89,8 @@ public class IncrementalSnapshotCheckBeforeRestoreTest extends AbstractSnapshotS /** */ @Test public void testCheckCorrectIncrementalSnapshot() throws Exception { + assert GRID_CNT > 2; + createFullSnapshot(); int incSnpCnt = 3; @@ -104,6 +106,29 @@ public void testCheckCorrectIncrementalSnapshot() throws Exception { assertTrue(res.idleVerifyResult().exceptions().isEmpty()); } } + + // Check from a smaller topology. + stopGrid(GRID_CNT); + + for (IgniteEx n : F.asList(grid(0), grid(GRID_CNT - 1))) { + for (int i = 0; i <= incSnpCnt; i++) { + SnapshotPartitionsVerifyResult res = snp(n).checkSnapshot(SNP, null, null, false, i, DFLT_CHECK_ON_RESTORE) + .get(getTestTimeout()); + + assertTrue(res.exceptions().isEmpty()); + assertTrue(res.idleVerifyResult().exceptions().isEmpty()); + } + } + + stopGrid(0); + + for (int i = 0; i <= incSnpCnt; i++) { + SnapshotPartitionsVerifyResult res = snp(grid(1)).checkSnapshot(SNP, null, null, false, i, DFLT_CHECK_ON_RESTORE) + .get(getTestTimeout()); + + assertTrue(res.exceptions().isEmpty()); + assertTrue(res.idleVerifyResult().exceptions().isEmpty()); + } } /** */ From 404cd6dcfae0eea531ead7b964238f606dd52b98 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 16 Oct 2024 16:15:49 +0300 Subject: [PATCH 14/27] fixes --- .../snapshot/SnapshotCheckProcess.java | 109 ++++++++++-------- .../persistence/snapshot/SnapshotChecker.java | 2 +- 2 files changed, 64 insertions(+), 47 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index da083b171ab57..64f0670b3ebc2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -150,7 +150,7 @@ private IgniteInternalFuture reduceValidatePartsAndFinish( return new GridFinishedFuture<>(); if (ctx.req.incrementalIndex() > 0) - reduceIncrementalCheckResults(ctx.req.nodes(), ctx.clusterMetas, results, errors, clusterOpFut); + reduceIncrementalResults(ctx.req.nodes(), ctx.clusterMetas, results, errors, clusterOpFut); else if (ctx.req.allRestoreHandlers()) reduceCustomHandlersResults(ctx, results, errors, clusterOpFut); else @@ -160,7 +160,7 @@ else if (ctx.req.allRestoreHandlers()) } /** */ - private void reduceIncrementalCheckResults( + private void reduceIncrementalResults( Set requiredNodes, Map> clusterMetas, Map results, @@ -175,16 +175,16 @@ private void reduceIncrementalCheckResults( UUID nodeId = resE.getKey(); SnapshotCheckIncrementalResponse incResp = (SnapshotCheckIncrementalResponse)resE.getValue(); - if (requiredNodes.contains(nodeId) || incResp == null) - break; + if (incResp == null || !requiredNodes.contains(nodeId)) + continue; ClusterNode node = kctx.cluster().get().node(nodeId); - incResp.incremenlatSnapshotResults().forEach((consId, incResPerConsId) -> - reduced.computeIfAbsent(node, nid -> new ArrayList<>()).add(incResPerConsId)); + incResp.incremenlatSnapshotResults().forEach((consId, res) -> reduced.computeIfAbsent(node, nid -> new ArrayList<>()) + .add(res)); if (F.isEmpty(incResp.exceptions())) - break; + continue; errors.putIfAbsent(nodeId, asException(F.firstValue(incResp.exceptions()))); } @@ -206,27 +206,31 @@ private void reduceCustomHandlersResults( throw F.firstValue(errors); // Check responses: node -> consistentId -> handler name -> handler result. - Map>>> cstRes = new HashMap<>(); + Map>>> reduced = new HashMap<>(); - for (Map.Entry respE : results.entrySet()) { - SnapshotCheckCustomHandlersResponse resp = (SnapshotCheckCustomHandlersResponse)respE.getValue(); + for (Map.Entry respEntry : results.entrySet()) { + SnapshotCheckCustomHandlersResponse nodeResp = (SnapshotCheckCustomHandlersResponse)respEntry.getValue(); - if (resp == null) - break; + if (nodeResp == null) + continue; - if (!F.isEmpty(resp.exceptions())) - throw F.firstValue(resp.exceptions()); + if (!F.isEmpty(nodeResp.exceptions())) + throw F.firstValue(nodeResp.exceptions()); - resp.customHandlersResults().forEach((consId, hndResMap) -> { - Map>> nodePerConsIdRes - = cstRes.computeIfAbsent(kctx.cluster().get().localNode(), n -> new HashMap<>()); + UUID nodeId = respEntry.getKey(); - hndResMap.forEach((hndId, hndRes) -> - nodePerConsIdRes.computeIfAbsent(consId, cstId -> new HashMap<>()).put(hndId, hndRes)); + // Node's response results: consistent id -> map of the handlers results per consistent id. + nodeResp.customHandlersResults().forEach((consId, respPerConsIdMap) -> { + // Reduced map of the handlers results per consistent id for certain node. + Map>> nodePerConsIdResultMap + = reduced.computeIfAbsent(kctx.cluster().get().node(nodeId), n -> new HashMap<>()); + + respPerConsIdMap.forEach((hndId, hndRes) -> + nodePerConsIdResultMap.computeIfAbsent(consId, cstId -> new HashMap<>()).put(hndId, hndRes)); }); } - kctx.cache().context().snapshotMgr().checker().checkCustomHandlersResults(ctx.req.snapshotName(), cstRes); + kctx.cache().context().snapshotMgr().checker().checkCustomHandlersResults(ctx.req.snapshotName(), reduced); fut.onDone(new SnapshotPartitionsVerifyResult(ctx.clusterMetas, null)); } @@ -245,28 +249,30 @@ private void reducePartitionsHashesResults( Map errors0 = mapErrors(errors); if (!results.isEmpty()) { - Map>> results0 = new HashMap<>(); + Map>> reduced = new HashMap<>(); - for (Map.Entry respE : results.entrySet()) { - UUID nodeId = respE.getKey(); - SnapshotCheckPartitionsHashesResponse resp = (SnapshotCheckPartitionsHashesResponse)respE.getValue(); + for (Map.Entry respEntry : results.entrySet()) { + SnapshotCheckPartitionsHashesResponse resp = (SnapshotCheckPartitionsHashesResponse)respEntry.getValue(); if (resp == null) - break; + continue; + + ClusterNode node = kctx.cluster().get().node(respEntry.getKey()); if (!F.isEmpty(resp.exceptions())) - errors0.putIfAbsent(kctx.cluster().get().node(nodeId), asException(F.firstValue(resp.exceptions()))); + errors0.putIfAbsent(node, asException(F.firstValue(resp.exceptions()))); - resp.partitionsHashes().forEach((consId, partsRes) -> { - Map> partsHashes - = results0.computeIfAbsent(kctx.cluster().get().localNode(), map -> new HashMap<>()); + // Partitions results map per consistent id for certain node responded. + resp.partitionsHashes().forEach((consId, partsMapPerConsId) -> { + // Reduced node's hashes on certain responded node for certain consistent id. + Map> nodeHashes = reduced.computeIfAbsent(node, map -> new HashMap<>()); - partsRes.forEach((partKey, partHash) -> partsHashes.computeIfAbsent(partKey, k -> new ArrayList<>()) + partsMapPerConsId.forEach((partKey, partHash) -> nodeHashes.computeIfAbsent(partKey, k -> new ArrayList<>()) .add(partHash)); }); } - IdleVerifyResultV2 chkRes = SnapshotChecker.reduceHashesResults(results0, errors0); + IdleVerifyResultV2 chkRes = SnapshotChecker.reduceHashesResults(reduced, errors0); fut.onDone(new SnapshotPartitionsVerifyResult(clusterMetas, chkRes)); } @@ -546,28 +552,39 @@ private void reducePreparationAndMetasCheck( } } - /** */ + /** + * Assigns metadatas to process. A snapshot can be checked on a smaller topology compared to the original one. In this case, + * some node has to check not only own partitions. + * + * @return Metadatas to process on current node if more than one found. + */ private List assingMetasToWork(Map> clusterMetas) { - List locMetas = clusterMetas.get(kctx.cluster().get().localNode()); + ClusterNode locNode = kctx.cluster().get().localNode(); + String locNodeConsIdStr = locNode.consistentId().toString(); + + List locMetas = clusterMetas.get(locNode); if (F.isEmpty(locMetas)) return null; - UUID minOrderDataNodeId = clusterMetas.keySet().stream().sorted(new Comparator<>() { - @Override public int compare(ClusterNode o1, ClusterNode o2) { - return Long.compare(o1.order(), o2.order()); - } - }).map(ClusterNode::id).findFirst().get(); + locMetas = new ArrayList<>(locMetas); - if (minOrderDataNodeId.equals(kctx.localNodeId())) { + UUID minOrderOfDataNode = clusterMetas.entrySet().stream().filter(e -> !F.isEmpty(e.getValue())) + .sorted(Comparator.comparingLong(e -> e.getKey().order())).findFirst().get().getKey().id(); + + if (minOrderOfDataNode.equals(locNode.id())) { Collection onlineDataNodesIds = clusterMetas.keySet().stream().map(node -> node.consistentId().toString()) .collect(Collectors.toSet()); - locMetas.removeIf(meta -> !meta.consistentId().equals(kctx.cluster().get().localNode().consistentId()) - && onlineDataNodesIds.contains(meta.consistentId())); + locMetas.removeIf(meta -> !meta.consistentId().equals(locNodeConsIdStr) && onlineDataNodesIds.contains(meta.consistentId())); + + assert locMetas.size() >= 1 : "Wrong number of metadatas to process found for current node."; + } + else { + locMetas = locMetas.stream().filter(meta -> meta.consistentId().equals(locNodeConsIdStr)).collect(Collectors.toList()); + + assert locMetas.size() == 1 : "No metadata found for current node to process"; } - else - locMetas = Collections.singletonList(F.first(locMetas)); return locMetas; } @@ -599,7 +616,7 @@ public IgniteInternalFuture start( Set requiredNodes = new HashSet<>(F.viewReadOnly(kctx.discovery().discoCache().aliveBaselineNodes(), F.node2id())); - // Initiator is also a required node. It collects the final oparation result. + // Initiator is also a required node. It collects the final operation result. requiredNodes.add(kctx.localNodeId()); SnapshotCheckProcessRequest req = new SnapshotCheckProcessRequest( @@ -755,13 +772,13 @@ private Map>> customHandlersRe } } - /** A DTO used to transfer incremental snapshot check results for phase 2. */ + /** A DTO used to transfer incremental snapshot check result for phase 2. */ private static final class SnapshotCheckIncrementalResponse extends AbstractSnapshotCheckResponse { /** Serial version uid. */ private static final long serialVersionUID = 0L; /** - * @param result Incremental snapshot check results per consistent id. + * @param result Incremental snapshot check result per consistent id. * @param exceptions Exceptions per consistent id. */ private SnapshotCheckIncrementalResponse( diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java index 5183fc7589b26..c8580e67ded6d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java @@ -1016,7 +1016,7 @@ public CompletableFuture> checkPartit } /** - * Checks results of the internal and custon snapshot validation handlres. Throws exception if a validation error occurs. + * Checks results of the internal and custom snapshot validation handlres. Throws exception if a validation error occurs. * @param snpName Snapshot name * @param results Results: checking node -> snapshot's node consistend id -> custom handler id -> handler result. * @see #invokeCustomHandlers(String, String, String, Collection, boolean) From cf9d40293f33951f037b407ff9eb2988e533e7cd Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 16 Oct 2024 18:58:36 +0300 Subject: [PATCH 15/27] fix --- .../persistence/snapshot/SnapshotCheckProcess.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index 64f0670b3ebc2..a2af32d42fb4d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -577,14 +577,14 @@ private List assingMetasToWork(Map !meta.consistentId().equals(locNodeConsIdStr) && onlineDataNodesIds.contains(meta.consistentId())); - - assert locMetas.size() >= 1 : "Wrong number of metadatas to process found for current node."; } - else { + else locMetas = locMetas.stream().filter(meta -> meta.consistentId().equals(locNodeConsIdStr)).collect(Collectors.toList()); - assert locMetas.size() == 1 : "No metadata found for current node to process"; - } + if (locMetas.isEmpty()) + locMetas = Collections.singletonList(F.first(clusterMetas.get(locNode))); + + assert locMetas.size() >= 1 : "Wrong number of metadatas to process found for current node."; return locMetas; } From 11740f039d2b456b7d211c0a7b1385318ac55a04 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 16 Oct 2024 20:54:02 +0300 Subject: [PATCH 16/27] reimpl metas collection --- .../snapshot/SnapshotCheckProcess.java | 46 ++++++++++++------- 1 file changed, 29 insertions(+), 17 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index a2af32d42fb4d..6fe3e1f63da2c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -558,35 +559,46 @@ private void reducePreparationAndMetasCheck( * * @return Metadatas to process on current node if more than one found. */ - private List assingMetasToWork(Map> clusterMetas) { + private @Nullable List assingMetasToWork(Map> clusterMetas) { ClusterNode locNode = kctx.cluster().get().localNode(); - String locNodeConsIdStr = locNode.consistentId().toString(); - List locMetas = clusterMetas.get(locNode); if (F.isEmpty(locMetas)) return null; - locMetas = new ArrayList<>(locMetas); + // Nodes sorted by lessser order. + Map> metasPerRespondedNodes = new HashMap<>(); - UUID minOrderOfDataNode = clusterMetas.entrySet().stream().filter(e -> !F.isEmpty(e.getValue())) - .sorted(Comparator.comparingLong(e -> e.getKey().order())).findFirst().get().getKey().id(); + Set onlineNodesConstIdsStr = new HashSet<>(clusterMetas.size()); - if (minOrderOfDataNode.equals(locNode.id())) { - Collection onlineDataNodesIds = clusterMetas.keySet().stream().map(node -> node.consistentId().toString()) - .collect(Collectors.toSet()); + clusterMetas.forEach((node, nodeMetas) -> { + if (!F.isEmpty(nodeMetas)) { + onlineNodesConstIdsStr.add(node.consistentId().toString()); - locMetas.removeIf(meta -> !meta.consistentId().equals(locNodeConsIdStr) && onlineDataNodesIds.contains(meta.consistentId())); - } - else - locMetas = locMetas.stream().filter(meta -> meta.consistentId().equals(locNodeConsIdStr)).collect(Collectors.toList()); + nodeMetas.forEach(nodeMeta -> metasPerRespondedNodes.computeIfAbsent(nodeMeta.consistentId(), + m -> new TreeSet<>(Comparator.comparingLong(ClusterNode::order))).add(node)); + } + }); + + String locNodeConsIdStr = locNode.consistentId().toString(); + + List metasToProc = new ArrayList<>(1); - if (locMetas.isEmpty()) - locMetas = Collections.singletonList(F.first(clusterMetas.get(locNode))); + for (SnapshotMetadata locMeta : locMetas) { + if (locMeta.consistentId().equals(locNodeConsIdStr)) { + assert !metasToProc.contains(locMeta) : "Local snapshot metadata is already assigned to process"; - assert locMetas.size() >= 1 : "Wrong number of metadatas to process found for current node."; + metasToProc.add(locMeta); + + continue; + } + + if (!onlineNodesConstIdsStr.contains(locMeta.consistentId()) + && F.first(metasPerRespondedNodes.get(locMeta.consistentId())).id().equals(kctx.localNodeId())) + metasToProc.add(locMeta); + } - return locMetas; + return metasToProc; } /** From e3b22e0a27376991f59fef66e1cea1421a3dee42 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Thu, 17 Oct 2024 19:09:59 +0300 Subject: [PATCH 17/27] merge fix --- .../snapshot/SnapshotCheckProcess.java | 115 ++++++++---------- .../persistence/snapshot/SnapshotChecker.java | 9 +- 2 files changed, 60 insertions(+), 64 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index 6fe3e1f63da2c..a6998d9a657d7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -172,9 +172,10 @@ private void reduceIncrementalResults( Map> reduced = new HashMap<>(); - for (Map.Entry resE : results.entrySet()) { - UUID nodeId = resE.getKey(); - SnapshotCheckIncrementalResponse incResp = (SnapshotCheckIncrementalResponse)resE.getValue(); + for (Map.Entry resEntry : results.entrySet()) { + UUID nodeId = resEntry.getKey(); + + SnapshotCheckIncrementalResponse incResp = (SnapshotCheckIncrementalResponse)resEntry.getValue(); if (incResp == null || !requiredNodes.contains(nodeId)) continue; @@ -206,7 +207,9 @@ private void reduceCustomHandlersResults( if (!errors.isEmpty()) throw F.firstValue(errors); - // Check responses: node -> consistentId -> handler name -> handler result. + SnapshotChecker snpChecker = kctx.cache().context().snapshotMgr().checker(); + + // Check responses: checking node -> snapshot part's consistent id -> handler name -> handler result. Map>>> reduced = new HashMap<>(); for (Map.Entry respEntry : results.entrySet()) { @@ -220,9 +223,8 @@ private void reduceCustomHandlersResults( UUID nodeId = respEntry.getKey(); - // Node's response results: consistent id -> map of the handlers results per consistent id. nodeResp.customHandlersResults().forEach((consId, respPerConsIdMap) -> { - // Reduced map of the handlers results per consistent id for certain node. + // Reduced map of the handlers results per snapshot part's consistent id for certain node. Map>> nodePerConsIdResultMap = reduced.computeIfAbsent(kctx.cluster().get().node(nodeId), n -> new HashMap<>()); @@ -231,7 +233,7 @@ private void reduceCustomHandlersResults( }); } - kctx.cache().context().snapshotMgr().checker().checkCustomHandlersResults(ctx.req.snapshotName(), reduced); + snpChecker.checkCustomHandlersResults(ctx.req.snapshotName(), reduced); fut.onDone(new SnapshotPartitionsVerifyResult(ctx.clusterMetas, null)); } @@ -263,7 +265,6 @@ private void reducePartitionsHashesResults( if (!F.isEmpty(resp.exceptions())) errors0.putIfAbsent(node, asException(F.firstValue(resp.exceptions()))); - // Partitions results map per consistent id for certain node responded. resp.partitionsHashes().forEach((consId, partsMapPerConsId) -> { // Reduced node's hashes on certain responded node for certain consistent id. Map> nodeHashes = reduced.computeIfAbsent(node, map -> new HashMap<>()); @@ -320,20 +321,18 @@ else if (req.allRestoreHandlers()) return phaseFut; } - /** @return A composed future of increment checks for each meta/consistent id regarding {@link SnapshotCheckContext#metas}. */ + /** @return A composed future of increment checks for each consistent id regarding {@link SnapshotCheckContext#metas}. */ private CompletableFuture incrementalFuture(SnapshotCheckContext ctx) { - assert !F.isEmpty(ctx.metas); - + SnapshotChecker snpChecker = kctx.cache().context().snapshotMgr().checker(); // Per metas result: consistent id -> check result. Map perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); - // Exceptions: consistent id -> exceptions. + // Per consistent id. Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); CompletableFuture composedFut = new CompletableFuture<>(); for (SnapshotMetadata meta : ctx.metas) { - CompletableFuture workingFut = kctx.cache().context().snapshotMgr().checker() - .checkIncrementalSnapshot(ctx.req.snapshotName(), + CompletableFuture workingFut = snpChecker.checkIncrementalSnapshot(ctx.req.snapshotName(), ctx.req.snapshotPath(), ctx.req.incrementalIndex()); workingFut.whenComplete((res, err) -> { @@ -350,17 +349,14 @@ private CompletableFuture incrementalFuture(Sn return composedFut; } - /** @return A composed future of partitions checks for each meta/consistent id regarding {@link SnapshotCheckContext#metas}. */ + /** @return A composed future of partitions checks for each consistent id regarding {@link SnapshotCheckContext#metas}. */ private CompletableFuture partitionsHashesFuture(SnapshotCheckContext ctx) { - assert !F.isEmpty(ctx.metas); - // Per metas result: consistent id -> check results per partition key. Map> perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); - // Exceptions: consistent id -> exceptions. + // Per consistent id. Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); CompletableFuture composedFut = new CompletableFuture<>(); AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); - IgniteSnapshotManager snpMgr = kctx.cache().context().snapshotMgr(); for (SnapshotMetadata meta : ctx.metas) { @@ -388,22 +384,21 @@ else if (!F.isEmpty(res)) } /** - * @return A composed future of all the snapshot handlers for each meta/consistent id regarding {@link SnapshotCheckContext#metas}. + * @return A composed future of all the snapshot handlers for each consistent id regarding {@link SnapshotCheckContext#metas}. * @see IgniteSnapshotManager#handlers() */ private CompletableFuture allHandlersFuture(SnapshotCheckContext ctx) { - assert !F.isEmpty(ctx.metas); - - // Per metas result: consistent id -> check result per handler id. + SnapshotChecker snpChecker = kctx.cache().context().snapshotMgr().checker(); + // Per metas result: snapshot part's consistent id -> check result per handler name. Map>> perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); - // Exceptions: consistent id -> exceptions. + // Per consistent id. Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); CompletableFuture composedFut = new CompletableFuture<>(); AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); for (SnapshotMetadata meta : ctx.metas) { - CompletableFuture>> metaFut = kctx.cache().context().snapshotMgr().checker() - .invokeCustomHandlers(meta, ctx.req.snapshotPath(), ctx.req.groups(), true); + CompletableFuture>> metaFut = snpChecker.invokeCustomHandlers(meta, + ctx.req.snapshotPath(), ctx.req.groups(), true); metaFut.whenComplete((res, err) -> { if (err != null) @@ -421,8 +416,8 @@ else if (!F.isEmpty(res)) /** */ private Map mapErrors(Map errors) { - return errors.entrySet().stream() - .collect(Collectors.toMap(e -> kctx.cluster().get().node(e.getKey()), e -> asException(e.getValue()))); + return errors.entrySet().stream().collect(Collectors.toMap(e -> kctx.cluster().get().node(e.getKey()), + e -> asException(e.getValue()))); } /** */ @@ -531,8 +526,8 @@ private void reducePreparationAndMetasCheck( if (!metasCheck.isEmpty()) throw new IgniteSnapshotVerifyException(metasCheck); - // If the topology is lesser that the snapshot's, we have to check partitions not only of current node. - ctx.metas = assingMetasToWork(metas); + // If the topology is lesser that the snapshot's, we have to check another partitions parts. + ctx.metas = assingMetas(metas); if (clusterOpFut != null) ctx.clusterMetas = metas; @@ -554,22 +549,21 @@ private void reducePreparationAndMetasCheck( } /** - * Assigns metadatas to process. A snapshot can be checked on a smaller topology compared to the original one. In this case, - * some node has to check not only own partitions. + * Assigns snapshot metadatas to process. A snapshot can be checked on a smaller topology compared to the original one. + * In this case, some node has to check not only own meta and partitions. * - * @return Metadatas to process on current node if more than one found. + * @return Metadatas to process on current node. */ - private @Nullable List assingMetasToWork(Map> clusterMetas) { + private @Nullable List assingMetas(Map> clusterMetas) { ClusterNode locNode = kctx.cluster().get().localNode(); List locMetas = clusterMetas.get(locNode); if (F.isEmpty(locMetas)) return null; - // Nodes sorted by lessser order. - Map> metasPerRespondedNodes = new HashMap<>(); - Set onlineNodesConstIdsStr = new HashSet<>(clusterMetas.size()); + // The nodes are sorted with lesser order. + Map> metasPerRespondedNodes = new HashMap<>(); clusterMetas.forEach((node, nodeMetas) -> { if (!F.isEmpty(nodeMetas)) { @@ -581,21 +575,20 @@ private void reducePreparationAndMetasCheck( }); String locNodeConsIdStr = locNode.consistentId().toString(); + List metasToProc = new ArrayList<>(); - List metasToProc = new ArrayList<>(1); - - for (SnapshotMetadata locMeta : locMetas) { - if (locMeta.consistentId().equals(locNodeConsIdStr)) { - assert !metasToProc.contains(locMeta) : "Local snapshot metadata is already assigned to process"; + for (SnapshotMetadata meta : locMetas) { + if (meta.consistentId().equals(locNodeConsIdStr)) { + assert !metasToProc.contains(meta) : "Local snapshot metadata is already assigned to process"; - metasToProc.add(locMeta); + metasToProc.add(meta); continue; } - if (!onlineNodesConstIdsStr.contains(locMeta.consistentId()) - && F.first(metasPerRespondedNodes.get(locMeta.consistentId())).id().equals(kctx.localNodeId())) - metasToProc.add(locMeta); + if (!onlineNodesConstIdsStr.contains(meta.consistentId()) + && F.first(metasPerRespondedNodes.get(meta.consistentId())).id().equals(kctx.localNodeId())) + metasToProc.add(meta); } return metasToProc; @@ -697,12 +690,12 @@ private GridFutureAdapter phaseFuture() { } } - /** A DTO base to transfer nodes' results for the both phases. */ + /** A DTO base to transfer node's results for the both phases. */ private abstract static class AbstractSnapshotCheckResponse implements Serializable { /** The result. */ protected Object result; - /** Exceptions per consistent id. */ + /** Exceptions per snapshot part's consistent id. */ @Nullable private Map exceptions; /** */ @@ -714,13 +707,13 @@ private AbstractSnapshotCheckResponse(Object result, @Nullable Map exceptions() { return exceptions; } } - /** A DTO to transfer metas result for phase 1. */ + /** A DTO to transfer snapshot metadatas result for phase 1. */ private static final class SnapshotCheckMetasResponse extends AbstractSnapshotCheckResponse { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -736,14 +729,14 @@ private List metas() { } } - /** A DTO used to transfer partition hashes result for phase 2. */ + /** A DTO to transfer partitionw hashes resultw for phase 2. */ private static final class SnapshotCheckPartitionsHashesResponse extends AbstractSnapshotCheckResponse { /** Serial version uid. */ private static final long serialVersionUID = 0L; /** - * @param result Partitions results per consistent id. - * @param exceptions Exceptions per consistent id. + * @param result Partitions results per snapshot part's consistent id. + * @param exceptions Exceptions per snapshot part's consistent id. */ private SnapshotCheckPartitionsHashesResponse( Map> result, @@ -752,14 +745,14 @@ private SnapshotCheckPartitionsHashesResponse( super(result, exceptions); } - /** @return Partitions hashes per consistent id. */ + /** @return Partitions hashes per snapshot part's consistent id. */ private Map> partitionsHashes() { return (Map>)result; } } /** - * A DTO used to transfer all the handlers results for phase 2. + * A DTO to transfer all the handlers results for phase 2. * * @see IgniteSnapshotManager#handlers(). */ @@ -768,8 +761,8 @@ private static final class SnapshotCheckCustomHandlersResponse extends AbstractS private static final long serialVersionUID = 0L; /** - * @param result Handlers results per consistent id. - * @param exceptions Exceptions per consistent id. + * @param result Handlers results per snapshot part's consistent id: consistent id -> handler name -> handler result. + * @param exceptions Exceptions per snapshot part's consistent id. */ private SnapshotCheckCustomHandlersResponse( Map>> result, @@ -778,7 +771,7 @@ private SnapshotCheckCustomHandlersResponse( super(result, exceptions); } - /** @return Handlers results per consistent id. */ + /** @return Handlers results per snapshot part's consistent id: consistent id -> handler name -> handler result. */ private Map>> customHandlersResults() { return (Map>>)result; } @@ -790,8 +783,8 @@ private static final class SnapshotCheckIncrementalResponse extends AbstractSnap private static final long serialVersionUID = 0L; /** - * @param result Incremental snapshot check result per consistent id. - * @param exceptions Exceptions per consistent id. + * @param result Incremental snapshot check result per snapshot part's consistent id. + * @param exceptions Exceptions per snapshot part's consistent id. */ private SnapshotCheckIncrementalResponse( Map result, @@ -800,7 +793,7 @@ private SnapshotCheckIncrementalResponse( super(result, exceptions); } - /** @return Incremental snapshot check results per consistent id. */ + /** @return Incremental snapshot check results per snapshot part's consistent id. */ private Map incremenlatSnapshotResults() { return (Map)result; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java index c8580e67ded6d..6d68396d9ebd0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java @@ -1016,9 +1016,9 @@ public CompletableFuture> checkPartit } /** - * Checks results of the internal and custom snapshot validation handlres. Throws exception if a validation error occurs. + * Checks results of all the snapshot validation handlres. * @param snpName Snapshot name - * @param results Results: checking node -> snapshot's node consistend id -> custom handler id -> handler result. + * @param results Results: checking node -> snapshot part's consistend id -> custom handler name -> handler result. * @see #invokeCustomHandlers(String, String, String, Collection, boolean) */ public void checkCustomHandlersResults( @@ -1028,15 +1028,18 @@ public void checkCustomHandlersResults( Map>> clusterResults = new HashMap<>(); Collection execNodes = new ArrayList<>(results.size()); + // Checking node -> Map by snapshot part's consistend id. for (Map.Entry>>> nodeRes : results.entrySet()) { + // Consistent id -> Map by handler name. for (Map.Entry>> nodeConsIdRes : nodeRes.getValue().entrySet()) { ClusterNode node = nodeRes.getKey(); - // Depending on the job mapping, we can get several different results from one node. + // We can get several different results from one node. execNodes.add(node.id()); assert nodeRes.getValue() != null : "At least the default snapshot restore handler should have been executed "; + // Handler name -> handler result. for (Map.Entry> nodeHndRes : nodeConsIdRes.getValue().entrySet()) { String hndName = nodeHndRes.getKey(); SnapshotHandlerResult hndRes = nodeHndRes.getValue(); From 836ce38f31264d744dec5fc964fbf67ca0bdad1c Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Thu, 17 Oct 2024 19:15:32 +0300 Subject: [PATCH 18/27] revert incremental snp check --- .../IgniteClusterSnapshotCheckTest.java | 2 +- ...ementalSnapshotCheckBeforeRestoreTest.java | 25 ------------------- 2 files changed, 1 insertion(+), 26 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java index 1cdb4d23adc7f..55eb57019f07e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java @@ -567,7 +567,7 @@ public void testClusterSnapshotCheckWithTwoCachesCheckNotCorrupted() throws Exce /** */ @Test public void testCheckFromLesserTopology() throws Exception { - // {@link #corruptPartitionFile} affetchs an encrypted partition wrongly. + // {@link #corruptPartitionFile} incorrectly affects an encrypted partition. assumeFalse(encryption); int srvCnt = 3; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java index fbd93f634b1ea..8723234381629 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/incremental/IncrementalSnapshotCheckBeforeRestoreTest.java @@ -89,8 +89,6 @@ public class IncrementalSnapshotCheckBeforeRestoreTest extends AbstractSnapshotS /** */ @Test public void testCheckCorrectIncrementalSnapshot() throws Exception { - assert GRID_CNT > 2; - createFullSnapshot(); int incSnpCnt = 3; @@ -106,29 +104,6 @@ public void testCheckCorrectIncrementalSnapshot() throws Exception { assertTrue(res.idleVerifyResult().exceptions().isEmpty()); } } - - // Check from a smaller topology. - stopGrid(GRID_CNT); - - for (IgniteEx n : F.asList(grid(0), grid(GRID_CNT - 1))) { - for (int i = 0; i <= incSnpCnt; i++) { - SnapshotPartitionsVerifyResult res = snp(n).checkSnapshot(SNP, null, null, false, i, DFLT_CHECK_ON_RESTORE) - .get(getTestTimeout()); - - assertTrue(res.exceptions().isEmpty()); - assertTrue(res.idleVerifyResult().exceptions().isEmpty()); - } - } - - stopGrid(0); - - for (int i = 0; i <= incSnpCnt; i++) { - SnapshotPartitionsVerifyResult res = snp(grid(1)).checkSnapshot(SNP, null, null, false, i, DFLT_CHECK_ON_RESTORE) - .get(getTestTimeout()); - - assertTrue(res.exceptions().isEmpty()); - assertTrue(res.idleVerifyResult().exceptions().isEmpty()); - } } /** */ From eca56f65f5f123c79f4fc1f20a580d6d7ce890be Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Fri, 27 Dec 2024 17:48:57 +0300 Subject: [PATCH 19/27] review fixes --- .../snapshot/SnapshotCheckProcess.java | 62 ++++++++----------- .../persistence/snapshot/SnapshotChecker.java | 14 ++++- 2 files changed, 38 insertions(+), 38 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index a6998d9a657d7..b5eeefe558036 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -182,8 +182,8 @@ private void reduceIncrementalResults( ClusterNode node = kctx.cluster().get().node(nodeId); - incResp.incremenlatSnapshotResults().forEach((consId, res) -> reduced.computeIfAbsent(node, nid -> new ArrayList<>()) - .add(res)); + // Incremental snapshot result. + incResp.result().forEach((consId, res) -> reduced.computeIfAbsent(node, nid -> new ArrayList<>()).add(res)); if (F.isEmpty(incResp.exceptions())) continue; @@ -223,7 +223,8 @@ private void reduceCustomHandlersResults( UUID nodeId = respEntry.getKey(); - nodeResp.customHandlersResults().forEach((consId, respPerConsIdMap) -> { + // Custom handlers' results. + nodeResp.result().forEach((consId, respPerConsIdMap) -> { // Reduced map of the handlers results per snapshot part's consistent id for certain node. Map>> nodePerConsIdResultMap = reduced.computeIfAbsent(kctx.cluster().get().node(nodeId), n -> new HashMap<>()); @@ -265,7 +266,8 @@ private void reducePartitionsHashesResults( if (!F.isEmpty(resp.exceptions())) errors0.putIfAbsent(node, asException(F.firstValue(resp.exceptions()))); - resp.partitionsHashes().forEach((consId, partsMapPerConsId) -> { + // Partitions hashes. + resp.result().forEach((consId, partsMapPerConsId) -> { // Reduced node's hashes on certain responded node for certain consistent id. Map> nodeHashes = reduced.computeIfAbsent(node, map -> new HashMap<>()); @@ -513,10 +515,10 @@ private void reducePreparationAndMetasCheck( results.forEach((nodeId, nodeRes) -> { // A node might be not required. It gives null result. But a required node might have invalid empty result // which must be validated. - if (ctx.req.nodes().contains(nodeId) && baseline(nodeId) && !F.isEmpty(nodeRes.metas())) { + if (ctx.req.nodes().contains(nodeId) && baseline(nodeId) && !F.isEmpty(nodeRes.result())) { assert nodeRes != null; - metas.put(kctx.cluster().get().node(nodeId), nodeRes.metas()); + metas.put(kctx.cluster().get().node(nodeId), nodeRes.result()); } }); @@ -691,15 +693,15 @@ private GridFutureAdapter phaseFuture() { } /** A DTO base to transfer node's results for the both phases. */ - private abstract static class AbstractSnapshotCheckResponse implements Serializable { + private abstract static class AbstractSnapshotCheckResponse implements Serializable { /** The result. */ - protected Object result; + protected T result; /** Exceptions per snapshot part's consistent id. */ - @Nullable private Map exceptions; + @Nullable private final Map exceptions; /** */ - private AbstractSnapshotCheckResponse(Object result, @Nullable Map exceptions) { + private AbstractSnapshotCheckResponse(T result, @Nullable Map exceptions) { assert result instanceof Serializable : "Snapshot check result is not serializable."; assert exceptions == null || exceptions instanceof Serializable : "Snapshot check exceptions aren't serializable."; @@ -708,13 +710,18 @@ private AbstractSnapshotCheckResponse(Object result, @Nullable Map exceptions() { + @Nullable Map exceptions() { return exceptions; } + + /** @return Certain phase's and process' result. */ + T result() { + return result; + } } /** A DTO to transfer snapshot metadatas result for phase 1. */ - private static final class SnapshotCheckMetasResponse extends AbstractSnapshotCheckResponse { + private static final class SnapshotCheckMetasResponse extends AbstractSnapshotCheckResponse> { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -722,15 +729,11 @@ private static final class SnapshotCheckMetasResponse extends AbstractSnapshotCh private SnapshotCheckMetasResponse(List result) { super(result, null); } - - /** @return All the snapshot metadatatas found on this node. */ - private List metas() { - return (List)result; - } } - /** A DTO to transfer partitionw hashes resultw for phase 2. */ - private static final class SnapshotCheckPartitionsHashesResponse extends AbstractSnapshotCheckResponse { + /** A DTO to transfer partition hashes result for phase 2. */ + private static final class SnapshotCheckPartitionsHashesResponse + extends AbstractSnapshotCheckResponse>> { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -744,11 +747,6 @@ private SnapshotCheckPartitionsHashesResponse( ) { super(result, exceptions); } - - /** @return Partitions hashes per snapshot part's consistent id. */ - private Map> partitionsHashes() { - return (Map>)result; - } } /** @@ -756,7 +754,8 @@ private Map> partitionsHashes * * @see IgniteSnapshotManager#handlers(). */ - private static final class SnapshotCheckCustomHandlersResponse extends AbstractSnapshotCheckResponse { + private static final class SnapshotCheckCustomHandlersResponse + extends AbstractSnapshotCheckResponse>>> { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -770,15 +769,11 @@ private SnapshotCheckCustomHandlersResponse( ) { super(result, exceptions); } - - /** @return Handlers results per snapshot part's consistent id: consistent id -> handler name -> handler result. */ - private Map>> customHandlersResults() { - return (Map>>)result; - } } /** A DTO used to transfer incremental snapshot check result for phase 2. */ - private static final class SnapshotCheckIncrementalResponse extends AbstractSnapshotCheckResponse { + private static final class SnapshotCheckIncrementalResponse + extends AbstractSnapshotCheckResponse> { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -792,10 +787,5 @@ private SnapshotCheckIncrementalResponse( ) { super(result, exceptions); } - - /** @return Incremental snapshot check results per snapshot part's consistent id. */ - private Map incremenlatSnapshotResults() { - return (Map)result; - } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java index 6d68396d9ebd0..44f7c60f9aea9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java @@ -639,8 +639,18 @@ public static IdleVerifyResultV2 reduceHashesResults( ) { Map> hashesRes = new HashMap<>(); - results.forEach((node, partsHashes) -> partsHashes.forEach((partKey, partHashLst) -> hashesRes.computeIfAbsent(partKey, - k -> new ArrayList<>()).addAll(partHashLst))); + // Iterate over node's results. + for (Map.Entry>> nodeHashes : results.entrySet()) { + Map> nodePartsHashes = nodeHashes.getValue(); + + // Iterate over partitions hashes related to the certain node. + for (Map.Entry> partHashes : nodePartsHashes.entrySet()) { + PartitionKeyV2 partKey = partHashes.getKey(); + List hashes = partHashes.getValue(); + + hashesRes.computeIfAbsent(partKey, k -> new ArrayList<>()).addAll(hashes); + } + } if (results.size() != ex.size()) return new IdleVerifyResultV2(hashesRes, ex); From 4a402e6c99681c536ee1a6859e54775a859ee863 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Sat, 28 Dec 2024 13:51:07 +0300 Subject: [PATCH 20/27] codestyle --- .../cache/VerifyBackupPartitionsTaskV2.java | 44 ++++++++----------- 1 file changed, 18 insertions(+), 26 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java index 83db900bdee32..dd2df139f1e47 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java @@ -112,7 +112,24 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter results) throws IgniteException { - return reduce0(results); + Map ex = new HashMap<>(); + Map>> hashes = new HashMap<>(); + + for (ComputeJobResult res : results) { + if (res.getException() != null) { + ex.put(res.getNode(), res.getException()); + + continue; + } + + Map> nodePartsRes = hashes.computeIfAbsent(res.getNode(), node -> new HashMap<>()); + + Map nodeData = res.getData(); + + nodeData.forEach((partKey, partHash) -> nodePartsRes.computeIfAbsent(partKey, k -> new ArrayList<>()).add(partHash)); + } + + return SnapshotChecker.reduceHashesResults(hashes, ex); } /** {@inheritDoc} */ @@ -140,31 +157,6 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter results) { - Map ex = new HashMap<>(); - Map>> hashes = new HashMap<>(); - - for (ComputeJobResult res : results) { - if (res.getException() != null) { - ex.put(res.getNode(), res.getException()); - - continue; - } - - Map> nodePartsRes = hashes.computeIfAbsent(res.getNode(), node -> new HashMap<>()); - - Map nodeData = res.getData(); - - nodeData.forEach((partKey, partHash) -> nodePartsRes.computeIfAbsent(partKey, k -> new ArrayList<>()).add(partHash)); - } - - return SnapshotChecker.reduceHashesResults(hashes, ex); - } - /** * Job that collects update counters and hashes of local partitions. */ From 86289607b09e8a9207c0665ad5c4edcc2f31ca10 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Sat, 28 Dec 2024 16:46:03 +0300 Subject: [PATCH 21/27] revert --- .../cache/VerifyBackupPartitionsTaskV2.java | 44 +++++++++++-------- 1 file changed, 26 insertions(+), 18 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java index dd2df139f1e47..83db900bdee32 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java @@ -112,24 +112,7 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter results) throws IgniteException { - Map ex = new HashMap<>(); - Map>> hashes = new HashMap<>(); - - for (ComputeJobResult res : results) { - if (res.getException() != null) { - ex.put(res.getNode(), res.getException()); - - continue; - } - - Map> nodePartsRes = hashes.computeIfAbsent(res.getNode(), node -> new HashMap<>()); - - Map nodeData = res.getData(); - - nodeData.forEach((partKey, partHash) -> nodePartsRes.computeIfAbsent(partKey, k -> new ArrayList<>()).add(partHash)); - } - - return SnapshotChecker.reduceHashesResults(hashes, ex); + return reduce0(results); } /** {@inheritDoc} */ @@ -157,6 +140,31 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter results) { + Map ex = new HashMap<>(); + Map>> hashes = new HashMap<>(); + + for (ComputeJobResult res : results) { + if (res.getException() != null) { + ex.put(res.getNode(), res.getException()); + + continue; + } + + Map> nodePartsRes = hashes.computeIfAbsent(res.getNode(), node -> new HashMap<>()); + + Map nodeData = res.getData(); + + nodeData.forEach((partKey, partHash) -> nodePartsRes.computeIfAbsent(partKey, k -> new ArrayList<>()).add(partHash)); + } + + return SnapshotChecker.reduceHashesResults(hashes, ex); + } + /** * Job that collects update counters and hashes of local partitions. */ From 05c923a7ab08e9166b8c95177e5b5eeafb39036e Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Sat, 28 Dec 2024 16:49:48 +0300 Subject: [PATCH 22/27] fix --- .../cache/persistence/snapshot/SnapshotChecker.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java index 44f7c60f9aea9..e42bb92c4c8ff 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java @@ -637,7 +637,7 @@ public static IdleVerifyResultV2 reduceHashesResults( Map>> results, Map ex ) { - Map> hashesRes = new HashMap<>(); + Map> clusterHashes = new HashMap<>(); // Iterate over node's results. for (Map.Entry>> nodeHashes : results.entrySet()) { @@ -648,12 +648,12 @@ public static IdleVerifyResultV2 reduceHashesResults( PartitionKeyV2 partKey = partHashes.getKey(); List hashes = partHashes.getValue(); - hashesRes.computeIfAbsent(partKey, k -> new ArrayList<>()).addAll(hashes); + clusterHashes.computeIfAbsent(partKey, k -> new ArrayList<>()).addAll(hashes); } } if (results.size() != ex.size()) - return new IdleVerifyResultV2(hashesRes, ex); + return new IdleVerifyResultV2(clusterHashes, ex); else return new IdleVerifyResultV2(ex); } From c8a3a807cd97b9fac9a26167da89a192a121437b Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Mon, 10 Feb 2025 16:53:19 +0300 Subject: [PATCH 23/27] review fix --- .../snapshot/SnapshotCheckProcess.java | 163 ++++++------------ 1 file changed, 50 insertions(+), 113 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index b5eeefe558036..0d382da363ec4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -72,10 +72,10 @@ public class SnapshotCheckProcess { private final Map> clusterOpFuts = new ConcurrentHashMap<>(); /** Check metas first phase subprocess. */ - private final DistributedProcess phase1CheckMetas; + private final DistributedProcess phase1CheckMetas; /** Partition hashes second phase subprocess. */ - private final DistributedProcess phase2PartsHashes; + private final DistributedProcess phase2PartsHashes; /** Stop node lock. */ private boolean nodeStopping; @@ -132,7 +132,7 @@ void interrupt(Throwable err) { /** Phase 2 and process finish. */ private IgniteInternalFuture reduceValidatePartsAndFinish( UUID reqId, - Map results, + Map results, Map errors ) { SnapshotCheckContext ctx = context(null, reqId); @@ -164,7 +164,7 @@ else if (ctx.req.allRestoreHandlers()) private void reduceIncrementalResults( Set requiredNodes, Map> clusterMetas, - Map results, + Map results, Map errors, GridFutureAdapter fut ) { @@ -172,18 +172,19 @@ private void reduceIncrementalResults( Map> reduced = new HashMap<>(); - for (Map.Entry resEntry : results.entrySet()) { + for (Map.Entry resEntry : results.entrySet()) { UUID nodeId = resEntry.getKey(); - SnapshotCheckIncrementalResponse incResp = (SnapshotCheckIncrementalResponse)resEntry.getValue(); + SnapshotCheckResponse incResp = resEntry.getValue(); if (incResp == null || !requiredNodes.contains(nodeId)) continue; ClusterNode node = kctx.cluster().get().node(nodeId); - // Incremental snapshot result. - incResp.result().forEach((consId, res) -> reduced.computeIfAbsent(node, nid -> new ArrayList<>()).add(res)); + Map incRes = incResp.result(); + + incRes.forEach((consId, res) -> reduced.computeIfAbsent(node, nid -> new ArrayList<>()).add(res)); if (F.isEmpty(incResp.exceptions())) continue; @@ -199,7 +200,7 @@ private void reduceIncrementalResults( /** */ private void reduceCustomHandlersResults( SnapshotCheckContext ctx, - Map results, + Map results, Map errors, GridFutureAdapter fut ) { @@ -212,8 +213,8 @@ private void reduceCustomHandlersResults( // Check responses: checking node -> snapshot part's consistent id -> handler name -> handler result. Map>>> reduced = new HashMap<>(); - for (Map.Entry respEntry : results.entrySet()) { - SnapshotCheckCustomHandlersResponse nodeResp = (SnapshotCheckCustomHandlersResponse)respEntry.getValue(); + for (Map.Entry respEntry : results.entrySet()) { + SnapshotCheckResponse nodeResp = respEntry.getValue(); if (nodeResp == null) continue; @@ -223,8 +224,9 @@ private void reduceCustomHandlersResults( UUID nodeId = respEntry.getKey(); - // Custom handlers' results. - nodeResp.result().forEach((consId, respPerConsIdMap) -> { + Map>> cstHndRes = nodeResp.result(); + + cstHndRes.forEach((consId, respPerConsIdMap) -> { // Reduced map of the handlers results per snapshot part's consistent id for certain node. Map>> nodePerConsIdResultMap = reduced.computeIfAbsent(kctx.cluster().get().node(nodeId), n -> new HashMap<>()); @@ -246,7 +248,7 @@ private void reduceCustomHandlersResults( /** */ private void reducePartitionsHashesResults( Map> clusterMetas, - Map results, + Map results, Map errors, GridFutureAdapter fut ) { @@ -255,8 +257,8 @@ private void reducePartitionsHashesResults( if (!results.isEmpty()) { Map>> reduced = new HashMap<>(); - for (Map.Entry respEntry : results.entrySet()) { - SnapshotCheckPartitionsHashesResponse resp = (SnapshotCheckPartitionsHashesResponse)respEntry.getValue(); + for (Map.Entry respEntry : results.entrySet()) { + SnapshotCheckResponse resp = respEntry.getValue(); if (resp == null) continue; @@ -266,8 +268,9 @@ private void reducePartitionsHashesResults( if (!F.isEmpty(resp.exceptions())) errors0.putIfAbsent(node, asException(F.firstValue(resp.exceptions()))); - // Partitions hashes. - resp.result().forEach((consId, partsMapPerConsId) -> { + Map> partsHashesRes = resp.result(); + + partsHashesRes.forEach((consId, partsMapPerConsId) -> { // Reduced node's hashes on certain responded node for certain consistent id. Map> nodeHashes = reduced.computeIfAbsent(node, map -> new HashMap<>()); @@ -285,7 +288,7 @@ private void reducePartitionsHashesResults( } /** Phase 2 beginning. */ - private IgniteInternalFuture validateParts(SnapshotCheckProcessRequest req) { + private IgniteInternalFuture validateParts(SnapshotCheckProcessRequest req) { if (!req.nodes().contains(kctx.localNodeId())) return new GridFinishedFuture<>(); @@ -296,11 +299,11 @@ private IgniteInternalFuture validateParts(Snapsh if (F.isEmpty(ctx.metas)) return new GridFinishedFuture<>(); - GridFutureAdapter phaseFut = ctx.phaseFuture(); + GridFutureAdapter phaseFut = ctx.phaseFuture(); // Might be already finished by asynchronous leave of a required node. if (!phaseFut.isDone()) { - CompletableFuture workingFut; + CompletableFuture workingFut; if (req.incrementalIndex() > 0) { assert !req.allRestoreHandlers() : "Snapshot handlers aren't supported for incremental snapshot."; @@ -324,14 +327,14 @@ else if (req.allRestoreHandlers()) } /** @return A composed future of increment checks for each consistent id regarding {@link SnapshotCheckContext#metas}. */ - private CompletableFuture incrementalFuture(SnapshotCheckContext ctx) { + private CompletableFuture incrementalFuture(SnapshotCheckContext ctx) { SnapshotChecker snpChecker = kctx.cache().context().snapshotMgr().checker(); // Per metas result: consistent id -> check result. Map perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); // Per consistent id. Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); - CompletableFuture composedFut = new CompletableFuture<>(); + CompletableFuture composedFut = new CompletableFuture<>(); for (SnapshotMetadata meta : ctx.metas) { CompletableFuture workingFut = snpChecker.checkIncrementalSnapshot(ctx.req.snapshotName(), @@ -344,7 +347,7 @@ private CompletableFuture incrementalFuture(Sn perMetaResults.put(meta.consistentId(), res); if (metasProcessed.decrementAndGet() == 0) - composedFut.complete( new SnapshotCheckIncrementalResponse(perMetaResults, exceptions)); + composedFut.complete( new SnapshotCheckResponse(perMetaResults, exceptions)); }); } @@ -352,12 +355,12 @@ private CompletableFuture incrementalFuture(Sn } /** @return A composed future of partitions checks for each consistent id regarding {@link SnapshotCheckContext#metas}. */ - private CompletableFuture partitionsHashesFuture(SnapshotCheckContext ctx) { + private CompletableFuture partitionsHashesFuture(SnapshotCheckContext ctx) { // Per metas result: consistent id -> check results per partition key. Map> perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); // Per consistent id. Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); - CompletableFuture composedFut = new CompletableFuture<>(); + CompletableFuture composedFut = new CompletableFuture<>(); AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); IgniteSnapshotManager snpMgr = kctx.cache().context().snapshotMgr(); @@ -378,7 +381,7 @@ else if (!F.isEmpty(res)) perMetaResults.put(meta.consistentId(), res); if (metasProcessed.decrementAndGet() == 0) - composedFut.complete(new SnapshotCheckPartitionsHashesResponse(perMetaResults, exceptions)); + composedFut.complete(new SnapshotCheckResponse(perMetaResults, exceptions)); }); } @@ -389,13 +392,13 @@ else if (!F.isEmpty(res)) * @return A composed future of all the snapshot handlers for each consistent id regarding {@link SnapshotCheckContext#metas}. * @see IgniteSnapshotManager#handlers() */ - private CompletableFuture allHandlersFuture(SnapshotCheckContext ctx) { + private CompletableFuture allHandlersFuture(SnapshotCheckContext ctx) { SnapshotChecker snpChecker = kctx.cache().context().snapshotMgr().checker(); // Per metas result: snapshot part's consistent id -> check result per handler name. Map>> perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); // Per consistent id. Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); - CompletableFuture composedFut = new CompletableFuture<>(); + CompletableFuture composedFut = new CompletableFuture<>(); AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); for (SnapshotMetadata meta : ctx.metas) { @@ -409,7 +412,7 @@ else if (!F.isEmpty(res)) perMetaResults.put(meta.consistentId(), res); if (metasProcessed.decrementAndGet() == 0) - composedFut.complete(new SnapshotCheckCustomHandlersResponse(perMetaResults, exceptions)); + composedFut.complete(new SnapshotCheckResponse(perMetaResults, exceptions)); }); } @@ -439,7 +442,7 @@ private static Exception asException(Throwable th) { } /** Phase 1 beginning: prepare, collect and check local metas. */ - private IgniteInternalFuture prepareAndCheckMetas(SnapshotCheckProcessRequest req) { + private IgniteInternalFuture prepareAndCheckMetas(SnapshotCheckProcessRequest req) { if (!req.nodes().contains(kctx.localNodeId())) return new GridFinishedFuture<>(); @@ -466,7 +469,7 @@ private IgniteInternalFuture prepareAndCheckMetas(Sn Collection grpIds = F.isEmpty(req.groups()) ? null : F.viewReadOnly(req.groups(), CU::cacheId); - GridFutureAdapter phaseFut = ctx.phaseFuture(); + GridFutureAdapter phaseFut = ctx.phaseFuture(); // Might be already finished by asynchronous leave of a required node. if (!phaseFut.isDone()) { @@ -479,7 +482,7 @@ private IgniteInternalFuture prepareAndCheckMetas(Sn if (err != null) phaseFut.onDone(err); else - phaseFut.onDone(new SnapshotCheckMetasResponse(locMetas)); + phaseFut.onDone(new SnapshotCheckResponse(locMetas, null)); }); } @@ -489,7 +492,7 @@ private IgniteInternalFuture prepareAndCheckMetas(Sn /** Phase 1 end. */ private void reducePreparationAndMetasCheck( UUID reqId, - Map results, + Map results, Map errors ) { SnapshotCheckContext ctx = context(null, reqId); @@ -515,7 +518,7 @@ private void reducePreparationAndMetasCheck( results.forEach((nodeId, nodeRes) -> { // A node might be not required. It gives null result. But a required node might have invalid empty result // which must be validated. - if (ctx.req.nodes().contains(nodeId) && baseline(nodeId) && !F.isEmpty(nodeRes.result())) { + if (ctx.req.nodes().contains(nodeId) && baseline(nodeId) && !F.isEmpty((Collection)nodeRes.result())) { assert nodeRes != null; metas.put(kctx.cluster().get().node(nodeId), nodeRes.result()); @@ -666,7 +669,7 @@ private static final class SnapshotCheckContext { private final SnapshotCheckProcessRequest req; /** Current process' future. Listens error, stop requests, etc. */ - private final GridFutureAdapter locProcFut = new GridFutureAdapter<>(); + private final GridFutureAdapter locProcFut = new GridFutureAdapter<>(); /** * Metadatas to process on this node. Also indicates the snapshot parts to check on this node. @@ -692,16 +695,19 @@ private GridFutureAdapter phaseFuture() { } } - /** A DTO base to transfer node's results for the both phases. */ - private abstract static class AbstractSnapshotCheckResponse implements Serializable { - /** The result. */ - protected T result; + /** A DTO to transfer node's results for the both phases. */ + private static final class SnapshotCheckResponse implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** The result. Is usually a collection or a map of hashes, metast, etc. */ + private final Object result; /** Exceptions per snapshot part's consistent id. */ @Nullable private final Map exceptions; /** */ - private AbstractSnapshotCheckResponse(T result, @Nullable Map exceptions) { + private SnapshotCheckResponse(Object result, @Nullable Map exceptions) { assert result instanceof Serializable : "Snapshot check result is not serializable."; assert exceptions == null || exceptions instanceof Serializable : "Snapshot check exceptions aren't serializable."; @@ -710,82 +716,13 @@ private AbstractSnapshotCheckResponse(T result, @Nullable Map } /** @return Exceptions per snapshot part's consistent id. */ - @Nullable Map exceptions() { + private @Nullable Map exceptions() { return exceptions; } /** @return Certain phase's and process' result. */ - T result() { - return result; - } - } - - /** A DTO to transfer snapshot metadatas result for phase 1. */ - private static final class SnapshotCheckMetasResponse extends AbstractSnapshotCheckResponse> { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** */ - private SnapshotCheckMetasResponse(List result) { - super(result, null); - } - } - - /** A DTO to transfer partition hashes result for phase 2. */ - private static final class SnapshotCheckPartitionsHashesResponse - extends AbstractSnapshotCheckResponse>> { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** - * @param result Partitions results per snapshot part's consistent id. - * @param exceptions Exceptions per snapshot part's consistent id. - */ - private SnapshotCheckPartitionsHashesResponse( - Map> result, - Map exceptions - ) { - super(result, exceptions); - } - } - - /** - * A DTO to transfer all the handlers results for phase 2. - * - * @see IgniteSnapshotManager#handlers(). - */ - private static final class SnapshotCheckCustomHandlersResponse - extends AbstractSnapshotCheckResponse>>> { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** - * @param result Handlers results per snapshot part's consistent id: consistent id -> handler name -> handler result. - * @param exceptions Exceptions per snapshot part's consistent id. - */ - private SnapshotCheckCustomHandlersResponse( - Map>> result, - Map exceptions - ) { - super(result, exceptions); - } - } - - /** A DTO used to transfer incremental snapshot check result for phase 2. */ - private static final class SnapshotCheckIncrementalResponse - extends AbstractSnapshotCheckResponse> { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** - * @param result Incremental snapshot check result per snapshot part's consistent id. - * @param exceptions Exceptions per snapshot part's consistent id. - */ - private SnapshotCheckIncrementalResponse( - Map result, - Map exceptions - ) { - super(result, exceptions); + private T result() { + return (T)result; } } } From d4e831982830a32fa255d958aeceafb925c72796 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Mon, 10 Feb 2025 16:53:19 +0300 Subject: [PATCH 24/27] refactor IdleVefiryRes --- .../management/cache/IdleVerifyResultV2.java | 131 +++++++++++-- .../cache/VerifyBackupPartitionsTaskV2.java | 14 +- .../snapshot/SnapshotCheckProcess.java | 182 ++++++------------ .../persistence/snapshot/SnapshotChecker.java | 51 +---- .../SnapshotPartitionsVerifyHandler.java | 15 +- 5 files changed, 189 insertions(+), 204 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java index f6dfb7afb7a83..5b885b8baba6d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java @@ -83,35 +83,23 @@ public class IdleVerifyResultV2 extends VisorDataTransferObject { * Default constructor for Externalizable. */ public IdleVerifyResultV2() { + // No-op. } /** - * @param exceptions Occurred exceptions. + * @see Builder */ - public IdleVerifyResultV2(Map exceptions) { + private IdleVerifyResultV2(Map exceptions) { this.exceptions = exceptions; } /** - * @param txHashConflicts Transaction hashes conflicts. + * @see Builder */ - public IdleVerifyResultV2( + private IdleVerifyResultV2( Map> clusterHashes, @Nullable List> txHashConflicts, - @Nullable Map> partiallyCommittedTxs - ) { - this(clusterHashes, Collections.emptyMap()); - - this.txHashConflicts = txHashConflicts; - this.partiallyCommittedTxs = partiallyCommittedTxs; - } - - /** - * @param clusterHashes Map of cluster partition hashes. - * @param exceptions Exceptions on each cluster node. - */ - public IdleVerifyResultV2( - Map> clusterHashes, + @Nullable Map> partiallyCommittedTxs, Map exceptions ) { for (Map.Entry> e : clusterHashes.entrySet()) { @@ -151,6 +139,8 @@ public IdleVerifyResultV2( } this.exceptions = exceptions; + this.txHashConflicts = txHashConflicts; + this.partiallyCommittedTxs = partiallyCommittedTxs; } /** {@inheritDoc} */ @@ -414,4 +404,109 @@ private void printConflicts(Consumer printer) { @Override public String toString() { return S.toString(IdleVerifyResultV2.class, this); } + + /** Creates a result with errors only. */ + public static IdleVerifyResultV2 errors(Map errors) { + return new IdleVerifyResultV2(errors); + } + + /** @return a fresh result builder. */ + public static Builder builder() { + return new Builder(); + } + + /** Check result builder. */ + public static final class Builder { + /** Holds data to build result in {@link #build()}. */ + private final IdleVerifyResultV2 hldr = new IdleVerifyResultV2(); + + /** */ + private Builder() { + // No-op. + } + + /** Build the final result. */ + public IdleVerifyResultV2 build() { + return new IdleVerifyResultV2(hldr.hashConflicts, hldr.txHashConflicts, hldr.partiallyCommittedTxs, hldr.exceptions); + } + + /** Stores an exception if nono is assigned for {@code node}.*/ + public Builder addException(ClusterNode node, Exception e) { + hldr.exceptions.putIfAbsent(node, e); + + return this; + } + + /** Sets the exceptions. */ + public Builder exceptions(Map ex) { + assert F.isEmpty(hldr.exceptions) : "Exceptions are already set."; + + hldr.exceptions = ex; + + return this; + } + + /** Stores a collection of partition hashes for partition key {@code key}. */ + public Builder addPartitionHashes(PartitionKeyV2 key, Collection hashes) { + // The conflicts are used just as a hash container. Are not conflicts at the monnent. + hldr.hashConflicts.compute(key, (key0, hashes0) -> { + if (hashes0 == null) + hashes0 = new ArrayList<>(); + + hashes0.addAll(hashes); + + return hashes0; + }); + + return this; + } + + /** Stores a partition hashes map. */ + public void addPartitionHashes(Map hashes) { + hashes.forEach((key, hash) -> addPartitionHashes(key, Collections.singletonList(hash))); + } + + /** Stores a single partition hash. */ + public Builder addPartitionHash(PartitionKeyV2 key, PartitionHashRecordV2 hash) { + addPartitionHashes(key, Collections.singletonList(hash)); + + return this; + } + + /** Sets partition hashes. */ + public Builder partitionHashes(Map> partHashes) { + assert F.isEmpty(hldr.hashConflicts) : "Partitiin hashes are arleadt set."; + + // The conflicts are used just as a hash container. Are not conflicts at the monnent. + hldr.hashConflicts = partHashes; + + return this; + } + + /** Adds transaction conflicts. */ + public Builder addTxConflicts(List txConflicts) { + hldr.txHashConflicts.add(txConflicts); + + return this; + } + + /** Adds partially commited transactions. */ + public Builder addPartiallyCommited(ClusterNode node, Collection versions) { + hldr.partiallyCommittedTxs.compute(node, (node0, versions0) -> { + if (versions0 == null) + versions0 = new ArrayList<>(); + + versions0.addAll(versions); + + return versions0; + }); + + return this; + } + + /** @return {@code True} if any error is stopre. {@code False} otherwise. */ + public boolean hasErrors() { + return !F.isEmpty(hldr.exceptions); + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java index 83db900bdee32..0ac11085c5702 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java @@ -52,7 +52,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotChecker; import org.apache.ignite.internal.processors.cache.verify.GridNotIdleException; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecordV2; import org.apache.ignite.internal.processors.task.GridInternal; @@ -145,24 +144,19 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter results) { - Map ex = new HashMap<>(); - Map>> hashes = new HashMap<>(); + IdleVerifyResultV2.Builder bldr = IdleVerifyResultV2.builder(); for (ComputeJobResult res : results) { if (res.getException() != null) { - ex.put(res.getNode(), res.getException()); + bldr.addException(res.getNode(), res.getException()); continue; } - Map> nodePartsRes = hashes.computeIfAbsent(res.getNode(), node -> new HashMap<>()); - - Map nodeData = res.getData(); - - nodeData.forEach((partKey, partHash) -> nodePartsRes.computeIfAbsent(partKey, k -> new ArrayList<>()).add(partHash)); + bldr.addPartitionHashes(res.getData()); } - return SnapshotChecker.reduceHashesResults(hashes, ex); + return bldr.build(); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index b5eeefe558036..a11ff41acb6f5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -72,10 +72,10 @@ public class SnapshotCheckProcess { private final Map> clusterOpFuts = new ConcurrentHashMap<>(); /** Check metas first phase subprocess. */ - private final DistributedProcess phase1CheckMetas; + private final DistributedProcess phase1CheckMetas; /** Partition hashes second phase subprocess. */ - private final DistributedProcess phase2PartsHashes; + private final DistributedProcess phase2PartsHashes; /** Stop node lock. */ private boolean nodeStopping; @@ -132,7 +132,7 @@ void interrupt(Throwable err) { /** Phase 2 and process finish. */ private IgniteInternalFuture reduceValidatePartsAndFinish( UUID reqId, - Map results, + Map results, Map errors ) { SnapshotCheckContext ctx = context(null, reqId); @@ -164,7 +164,7 @@ else if (ctx.req.allRestoreHandlers()) private void reduceIncrementalResults( Set requiredNodes, Map> clusterMetas, - Map results, + Map results, Map errors, GridFutureAdapter fut ) { @@ -172,18 +172,19 @@ private void reduceIncrementalResults( Map> reduced = new HashMap<>(); - for (Map.Entry resEntry : results.entrySet()) { + for (Map.Entry resEntry : results.entrySet()) { UUID nodeId = resEntry.getKey(); - SnapshotCheckIncrementalResponse incResp = (SnapshotCheckIncrementalResponse)resEntry.getValue(); + SnapshotCheckResponse incResp = resEntry.getValue(); if (incResp == null || !requiredNodes.contains(nodeId)) continue; ClusterNode node = kctx.cluster().get().node(nodeId); - // Incremental snapshot result. - incResp.result().forEach((consId, res) -> reduced.computeIfAbsent(node, nid -> new ArrayList<>()).add(res)); + Map incRes = incResp.result(); + + incRes.forEach((consId, res) -> reduced.computeIfAbsent(node, nid -> new ArrayList<>()).add(res)); if (F.isEmpty(incResp.exceptions())) continue; @@ -199,7 +200,7 @@ private void reduceIncrementalResults( /** */ private void reduceCustomHandlersResults( SnapshotCheckContext ctx, - Map results, + Map results, Map errors, GridFutureAdapter fut ) { @@ -212,8 +213,8 @@ private void reduceCustomHandlersResults( // Check responses: checking node -> snapshot part's consistent id -> handler name -> handler result. Map>>> reduced = new HashMap<>(); - for (Map.Entry respEntry : results.entrySet()) { - SnapshotCheckCustomHandlersResponse nodeResp = (SnapshotCheckCustomHandlersResponse)respEntry.getValue(); + for (Map.Entry respEntry : results.entrySet()) { + SnapshotCheckResponse nodeResp = respEntry.getValue(); if (nodeResp == null) continue; @@ -223,8 +224,9 @@ private void reduceCustomHandlersResults( UUID nodeId = respEntry.getKey(); - // Custom handlers' results. - nodeResp.result().forEach((consId, respPerConsIdMap) -> { + Map>> cstHndRes = nodeResp.result(); + + cstHndRes.forEach((consId, respPerConsIdMap) -> { // Reduced map of the handlers results per snapshot part's consistent id for certain node. Map>> nodePerConsIdResultMap = reduced.computeIfAbsent(kctx.cluster().get().node(nodeId), n -> new HashMap<>()); @@ -246,46 +248,42 @@ private void reduceCustomHandlersResults( /** */ private void reducePartitionsHashesResults( Map> clusterMetas, - Map results, + Map results, Map errors, GridFutureAdapter fut ) { + IdleVerifyResultV2.Builder bldr = IdleVerifyResultV2.builder(); + Map errors0 = mapErrors(errors); if (!results.isEmpty()) { - Map>> reduced = new HashMap<>(); + bldr.exceptions(mapErrors(errors)); - for (Map.Entry respEntry : results.entrySet()) { - SnapshotCheckPartitionsHashesResponse resp = (SnapshotCheckPartitionsHashesResponse)respEntry.getValue(); + for (Map.Entry respEntry : results.entrySet()) { + SnapshotCheckResponse resp = respEntry.getValue(); if (resp == null) continue; - ClusterNode node = kctx.cluster().get().node(respEntry.getKey()); + if (!F.isEmpty(resp.exceptions())) { + ClusterNode node = kctx.cluster().get().node(respEntry.getKey()); - if (!F.isEmpty(resp.exceptions())) - errors0.putIfAbsent(node, asException(F.firstValue(resp.exceptions()))); + bldr.addException(node, asException(F.firstValue(resp.exceptions()))); + } - // Partitions hashes. - resp.result().forEach((consId, partsMapPerConsId) -> { - // Reduced node's hashes on certain responded node for certain consistent id. - Map> nodeHashes = reduced.computeIfAbsent(node, map -> new HashMap<>()); + Map> partsHashesRes = resp.result(); - partsMapPerConsId.forEach((partKey, partHash) -> nodeHashes.computeIfAbsent(partKey, k -> new ArrayList<>()) - .add(partHash)); - }); + partsHashesRes.forEach((consId, partsPerConsId) -> bldr.addPartitionHashes(partsPerConsId)); } - IdleVerifyResultV2 chkRes = SnapshotChecker.reduceHashesResults(reduced, errors0); - - fut.onDone(new SnapshotPartitionsVerifyResult(clusterMetas, chkRes)); + fut.onDone(new SnapshotPartitionsVerifyResult(clusterMetas, bldr.build())); } else fut.onDone(new IgniteSnapshotVerifyException(errors0)); } /** Phase 2 beginning. */ - private IgniteInternalFuture validateParts(SnapshotCheckProcessRequest req) { + private IgniteInternalFuture validateParts(SnapshotCheckProcessRequest req) { if (!req.nodes().contains(kctx.localNodeId())) return new GridFinishedFuture<>(); @@ -296,11 +294,11 @@ private IgniteInternalFuture validateParts(Snapsh if (F.isEmpty(ctx.metas)) return new GridFinishedFuture<>(); - GridFutureAdapter phaseFut = ctx.phaseFuture(); + GridFutureAdapter phaseFut = ctx.phaseFuture(); // Might be already finished by asynchronous leave of a required node. if (!phaseFut.isDone()) { - CompletableFuture workingFut; + CompletableFuture workingFut; if (req.incrementalIndex() > 0) { assert !req.allRestoreHandlers() : "Snapshot handlers aren't supported for incremental snapshot."; @@ -324,14 +322,14 @@ else if (req.allRestoreHandlers()) } /** @return A composed future of increment checks for each consistent id regarding {@link SnapshotCheckContext#metas}. */ - private CompletableFuture incrementalFuture(SnapshotCheckContext ctx) { + private CompletableFuture incrementalFuture(SnapshotCheckContext ctx) { SnapshotChecker snpChecker = kctx.cache().context().snapshotMgr().checker(); // Per metas result: consistent id -> check result. Map perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); // Per consistent id. Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); - CompletableFuture composedFut = new CompletableFuture<>(); + CompletableFuture composedFut = new CompletableFuture<>(); for (SnapshotMetadata meta : ctx.metas) { CompletableFuture workingFut = snpChecker.checkIncrementalSnapshot(ctx.req.snapshotName(), @@ -344,7 +342,7 @@ private CompletableFuture incrementalFuture(Sn perMetaResults.put(meta.consistentId(), res); if (metasProcessed.decrementAndGet() == 0) - composedFut.complete( new SnapshotCheckIncrementalResponse(perMetaResults, exceptions)); + composedFut.complete( new SnapshotCheckResponse(perMetaResults, exceptions)); }); } @@ -352,12 +350,12 @@ private CompletableFuture incrementalFuture(Sn } /** @return A composed future of partitions checks for each consistent id regarding {@link SnapshotCheckContext#metas}. */ - private CompletableFuture partitionsHashesFuture(SnapshotCheckContext ctx) { + private CompletableFuture partitionsHashesFuture(SnapshotCheckContext ctx) { // Per metas result: consistent id -> check results per partition key. Map> perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); // Per consistent id. Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); - CompletableFuture composedFut = new CompletableFuture<>(); + CompletableFuture composedFut = new CompletableFuture<>(); AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); IgniteSnapshotManager snpMgr = kctx.cache().context().snapshotMgr(); @@ -378,7 +376,7 @@ else if (!F.isEmpty(res)) perMetaResults.put(meta.consistentId(), res); if (metasProcessed.decrementAndGet() == 0) - composedFut.complete(new SnapshotCheckPartitionsHashesResponse(perMetaResults, exceptions)); + composedFut.complete(new SnapshotCheckResponse(perMetaResults, exceptions)); }); } @@ -389,13 +387,13 @@ else if (!F.isEmpty(res)) * @return A composed future of all the snapshot handlers for each consistent id regarding {@link SnapshotCheckContext#metas}. * @see IgniteSnapshotManager#handlers() */ - private CompletableFuture allHandlersFuture(SnapshotCheckContext ctx) { + private CompletableFuture allHandlersFuture(SnapshotCheckContext ctx) { SnapshotChecker snpChecker = kctx.cache().context().snapshotMgr().checker(); // Per metas result: snapshot part's consistent id -> check result per handler name. Map>> perMetaResults = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); // Per consistent id. Map exceptions = new ConcurrentHashMap<>(ctx.metas.size(), 1.0f); - CompletableFuture composedFut = new CompletableFuture<>(); + CompletableFuture composedFut = new CompletableFuture<>(); AtomicInteger metasProcessed = new AtomicInteger(ctx.metas.size()); for (SnapshotMetadata meta : ctx.metas) { @@ -409,7 +407,7 @@ else if (!F.isEmpty(res)) perMetaResults.put(meta.consistentId(), res); if (metasProcessed.decrementAndGet() == 0) - composedFut.complete(new SnapshotCheckCustomHandlersResponse(perMetaResults, exceptions)); + composedFut.complete(new SnapshotCheckResponse(perMetaResults, exceptions)); }); } @@ -439,7 +437,7 @@ private static Exception asException(Throwable th) { } /** Phase 1 beginning: prepare, collect and check local metas. */ - private IgniteInternalFuture prepareAndCheckMetas(SnapshotCheckProcessRequest req) { + private IgniteInternalFuture prepareAndCheckMetas(SnapshotCheckProcessRequest req) { if (!req.nodes().contains(kctx.localNodeId())) return new GridFinishedFuture<>(); @@ -466,7 +464,7 @@ private IgniteInternalFuture prepareAndCheckMetas(Sn Collection grpIds = F.isEmpty(req.groups()) ? null : F.viewReadOnly(req.groups(), CU::cacheId); - GridFutureAdapter phaseFut = ctx.phaseFuture(); + GridFutureAdapter phaseFut = ctx.phaseFuture(); // Might be already finished by asynchronous leave of a required node. if (!phaseFut.isDone()) { @@ -479,7 +477,7 @@ private IgniteInternalFuture prepareAndCheckMetas(Sn if (err != null) phaseFut.onDone(err); else - phaseFut.onDone(new SnapshotCheckMetasResponse(locMetas)); + phaseFut.onDone(new SnapshotCheckResponse(locMetas, null)); }); } @@ -489,7 +487,7 @@ private IgniteInternalFuture prepareAndCheckMetas(Sn /** Phase 1 end. */ private void reducePreparationAndMetasCheck( UUID reqId, - Map results, + Map results, Map errors ) { SnapshotCheckContext ctx = context(null, reqId); @@ -515,7 +513,7 @@ private void reducePreparationAndMetasCheck( results.forEach((nodeId, nodeRes) -> { // A node might be not required. It gives null result. But a required node might have invalid empty result // which must be validated. - if (ctx.req.nodes().contains(nodeId) && baseline(nodeId) && !F.isEmpty(nodeRes.result())) { + if (ctx.req.nodes().contains(nodeId) && baseline(nodeId) && !F.isEmpty((Collection)nodeRes.result())) { assert nodeRes != null; metas.put(kctx.cluster().get().node(nodeId), nodeRes.result()); @@ -666,7 +664,7 @@ private static final class SnapshotCheckContext { private final SnapshotCheckProcessRequest req; /** Current process' future. Listens error, stop requests, etc. */ - private final GridFutureAdapter locProcFut = new GridFutureAdapter<>(); + private final GridFutureAdapter locProcFut = new GridFutureAdapter<>(); /** * Metadatas to process on this node. Also indicates the snapshot parts to check on this node. @@ -692,16 +690,19 @@ private GridFutureAdapter phaseFuture() { } } - /** A DTO base to transfer node's results for the both phases. */ - private abstract static class AbstractSnapshotCheckResponse implements Serializable { - /** The result. */ - protected T result; + /** A DTO to transfer node's results for the both phases. */ + private static final class SnapshotCheckResponse implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** The result. Is usually a collection or a map of hashes, metast, etc. */ + private final Object result; /** Exceptions per snapshot part's consistent id. */ @Nullable private final Map exceptions; /** */ - private AbstractSnapshotCheckResponse(T result, @Nullable Map exceptions) { + private SnapshotCheckResponse(Object result, @Nullable Map exceptions) { assert result instanceof Serializable : "Snapshot check result is not serializable."; assert exceptions == null || exceptions instanceof Serializable : "Snapshot check exceptions aren't serializable."; @@ -710,82 +711,13 @@ private AbstractSnapshotCheckResponse(T result, @Nullable Map } /** @return Exceptions per snapshot part's consistent id. */ - @Nullable Map exceptions() { + private @Nullable Map exceptions() { return exceptions; } /** @return Certain phase's and process' result. */ - T result() { - return result; - } - } - - /** A DTO to transfer snapshot metadatas result for phase 1. */ - private static final class SnapshotCheckMetasResponse extends AbstractSnapshotCheckResponse> { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** */ - private SnapshotCheckMetasResponse(List result) { - super(result, null); - } - } - - /** A DTO to transfer partition hashes result for phase 2. */ - private static final class SnapshotCheckPartitionsHashesResponse - extends AbstractSnapshotCheckResponse>> { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** - * @param result Partitions results per snapshot part's consistent id. - * @param exceptions Exceptions per snapshot part's consistent id. - */ - private SnapshotCheckPartitionsHashesResponse( - Map> result, - Map exceptions - ) { - super(result, exceptions); - } - } - - /** - * A DTO to transfer all the handlers results for phase 2. - * - * @see IgniteSnapshotManager#handlers(). - */ - private static final class SnapshotCheckCustomHandlersResponse - extends AbstractSnapshotCheckResponse>>> { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** - * @param result Handlers results per snapshot part's consistent id: consistent id -> handler name -> handler result. - * @param exceptions Exceptions per snapshot part's consistent id. - */ - private SnapshotCheckCustomHandlersResponse( - Map>> result, - Map exceptions - ) { - super(result, exceptions); - } - } - - /** A DTO used to transfer incremental snapshot check result for phase 2. */ - private static final class SnapshotCheckIncrementalResponse - extends AbstractSnapshotCheckResponse> { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** - * @param result Incremental snapshot check result per snapshot part's consistent id. - * @param exceptions Exceptions per snapshot part's consistent id. - */ - private SnapshotCheckIncrementalResponse( - Map result, - Map exceptions - ) { - super(result, exceptions); + private T result() { + return (T)result; } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java index e42bb92c4c8ff..35c84668a6b88 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java @@ -579,21 +579,18 @@ public IdleVerifyResultV2 reduceIncrementalResults( Map operationErrors ) { if (!operationErrors.isEmpty()) - return new IdleVerifyResultV2(operationErrors); + return IdleVerifyResultV2.errors(operationErrors); + + IdleVerifyResultV2.Builder bldr = IdleVerifyResultV2.builder(); Map> nodeTxHashMap = new HashMap<>(); - List> txHashConflicts = new ArrayList<>(); - Map> partHashes = new HashMap<>(); - Map> partiallyCommittedTxs = new HashMap<>(); - Map errors = new HashMap<>(); results.forEach((node, resLst) -> resLst.forEach(res -> { - if (res.exceptions().isEmpty() && errors.isEmpty()) { + if (res.exceptions().isEmpty() && !bldr.hasErrors()) { if (!F.isEmpty(res.partiallyCommittedTxs())) - partiallyCommittedTxs.put(node, res.partiallyCommittedTxs()); + bldr.addPartiallyCommited(node, res.partiallyCommittedTxs()); - for (Map.Entry entry : res.partHashRes().entrySet()) - partHashes.computeIfAbsent(entry.getKey(), v -> new ArrayList<>()).add(entry.getValue()); + bldr.addPartitionHashes(res.partHashRes()); if (log.isDebugEnabled()) log.debug("Handle VerifyIncrementalSnapshotJob result [node=" + node + ", taskRes=" + res + ']'); @@ -612,50 +609,22 @@ public IdleVerifyResultV2 reduceIncrementalResults( TransactionsHashRecord prevHash = prevNodeTxHash.remove(hash.localConsistentId()); if (prevHash == null || prevHash.transactionHash() != hash.transactionHash()) - txHashConflicts.add(F.asList(hash, prevHash)); + bldr.addTxConflicts(F.asList(hash, prevHash)); resIt.remove(); } } } else if (!res.exceptions().isEmpty()) - errors.put(node, F.first(res.exceptions())); + bldr.addException(node, F.first(res.exceptions())); })); // Add all missed pairs to conflicts. nodeTxHashMap.values().stream() .flatMap(e -> e.values().stream()) - .forEach(e -> txHashConflicts.add(F.asList(e, null))); - - return errors.isEmpty() - ? new IdleVerifyResultV2(partHashes, txHashConflicts, partiallyCommittedTxs) - : new IdleVerifyResultV2(errors); - } - - /** */ - public static IdleVerifyResultV2 reduceHashesResults( - Map>> results, - Map ex - ) { - Map> clusterHashes = new HashMap<>(); - - // Iterate over node's results. - for (Map.Entry>> nodeHashes : results.entrySet()) { - Map> nodePartsHashes = nodeHashes.getValue(); - - // Iterate over partitions hashes related to the certain node. - for (Map.Entry> partHashes : nodePartsHashes.entrySet()) { - PartitionKeyV2 partKey = partHashes.getKey(); - List hashes = partHashes.getValue(); - - clusterHashes.computeIfAbsent(partKey, k -> new ArrayList<>()).addAll(hashes); - } - } + .forEach(e -> bldr.addTxConflicts(F.asList(e, null))); - if (results.size() != ex.size()) - return new IdleVerifyResultV2(clusterHashes, ex); - else - return new IdleVerifyResultV2(ex); + return bldr.build(); } /** */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java index 78db07b74fd8b..817866fb3f69f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java @@ -17,14 +17,10 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; -import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; -import java.util.List; import java.util.Map; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; -import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; import org.apache.ignite.internal.management.cache.PartitionKeyV2; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; @@ -63,23 +59,22 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext cctx) { /** {@inheritDoc} */ @Override public void complete(String name, Collection>> results) throws IgniteCheckedException { - Map> clusterHashes = new HashMap<>(); - Map errs = new HashMap<>(); + IdleVerifyResultV2.Builder bldr = IdleVerifyResultV2.builder(); for (SnapshotHandlerResult> res : results) { if (res.error() != null) { - errs.put(res.node(), res.error()); + bldr.addException(res.node(), res.error()); continue; } for (Map.Entry entry : res.data().entrySet()) - clusterHashes.computeIfAbsent(entry.getKey(), v -> new ArrayList<>()).add(entry.getValue()); + bldr.addPartitionHash(entry.getKey(), entry.getValue()); } - IdleVerifyResultV2 verifyResult = new IdleVerifyResultV2(clusterHashes, errs); + IdleVerifyResultV2 verifyResult = bldr.build(); - if (errs.isEmpty() && !verifyResult.hasConflicts()) + if (!bldr.hasErrors() && !verifyResult.hasConflicts()) return; GridStringBuilder buf = new GridStringBuilder(); From afb9b9d7f82af200185cc105a46d52783d098481 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Tue, 11 Feb 2025 16:52:02 +0300 Subject: [PATCH 25/27] minor --- .../internal/management/cache/IdleVerifyResultV2.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java index 5b885b8baba6d..dbd3fe2f50344 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java @@ -77,7 +77,7 @@ public class IdleVerifyResultV2 extends VisorDataTransferObject { /** Exceptions. */ @GridToStringInclude - private Map exceptions; + @Nullable private Map exceptions; /** * Default constructor for Externalizable. @@ -100,7 +100,7 @@ private IdleVerifyResultV2( Map> clusterHashes, @Nullable List> txHashConflicts, @Nullable Map> partiallyCommittedTxs, - Map exceptions + @Nullable Map exceptions ) { for (Map.Entry> e : clusterHashes.entrySet()) { Integer partHash = null; @@ -217,7 +217,7 @@ public boolean hasConflicts() { /** * @return Exceptions on nodes. */ - public Map exceptions() { + @Nullable public Map exceptions() { return exceptions; } @@ -432,6 +432,8 @@ public IdleVerifyResultV2 build() { /** Stores an exception if nono is assigned for {@code node}.*/ public Builder addException(ClusterNode node, Exception e) { + assert e != null; + hldr.exceptions.putIfAbsent(node, e); return this; From 6dc945f02f4715e2745ce352b21347c14c545294 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Tue, 11 Feb 2025 17:24:01 +0300 Subject: [PATCH 26/27] minor refactor --- .../management/cache/IdleVerifyResultV2.java | 94 ++++++++++++------- .../persistence/snapshot/SnapshotChecker.java | 2 +- 2 files changed, 59 insertions(+), 37 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java index dbd3fe2f50344..b2944e0a52548 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java @@ -77,7 +77,7 @@ public class IdleVerifyResultV2 extends VisorDataTransferObject { /** Exceptions. */ @GridToStringInclude - @Nullable private Map exceptions; + private Map exceptions; /** * Default constructor for Externalizable. @@ -100,7 +100,7 @@ private IdleVerifyResultV2( Map> clusterHashes, @Nullable List> txHashConflicts, @Nullable Map> partiallyCommittedTxs, - @Nullable Map exceptions + Map exceptions ) { for (Map.Entry> e : clusterHashes.entrySet()) { Integer partHash = null; @@ -217,7 +217,7 @@ public boolean hasConflicts() { /** * @return Exceptions on nodes. */ - @Nullable public Map exceptions() { + public Map exceptions() { return exceptions; } @@ -405,20 +405,24 @@ private void printConflicts(Consumer printer) { return S.toString(IdleVerifyResultV2.class, this); } - /** Creates a result with errors only. */ - public static IdleVerifyResultV2 errors(Map errors) { - return new IdleVerifyResultV2(errors); - } - - /** @return a fresh result builder. */ + /** @return A fresh result builder. */ public static Builder builder() { return new Builder(); } - /** Check result builder. */ + /** Builder of {@link IdleVerifyResultV2}. Is not thread-safe. */ public static final class Builder { - /** Holds data to build result in {@link #build()}. */ - private final IdleVerifyResultV2 hldr = new IdleVerifyResultV2(); + /** */ + private @Nullable Map> partHashes; + + /** */ + private @Nullable List> txHashConflicts; + + /** */ + private @Nullable Map> partiallyCommittedTxs; + + /** */ + private Map exceptions; /** */ private Builder() { @@ -427,35 +431,47 @@ private Builder() { /** Build the final result. */ public IdleVerifyResultV2 build() { - return new IdleVerifyResultV2(hldr.hashConflicts, hldr.txHashConflicts, hldr.partiallyCommittedTxs, hldr.exceptions); + if (partHashes == null) + partHashes = Collections.emptyMap(); + + if (exceptions == null) + exceptions = Collections.emptyMap(); + + return new IdleVerifyResultV2(partHashes, txHashConflicts, partiallyCommittedTxs, exceptions); } - /** Stores an exception if nono is assigned for {@code node}.*/ + /** Stores an exception if none is assigned for {@code node}. */ public Builder addException(ClusterNode node, Exception e) { assert e != null; - hldr.exceptions.putIfAbsent(node, e); + if (exceptions == null) + exceptions = new HashMap<>(); + + exceptions.putIfAbsent(node, e); return this; } /** Sets the exceptions. */ - public Builder exceptions(Map ex) { - assert F.isEmpty(hldr.exceptions) : "Exceptions are already set."; + public Builder exceptions(Map exceptions) { + assert this.exceptions == null : "Exceptions are already set."; + assert exceptions != null; - hldr.exceptions = ex; + this.exceptions = exceptions; return this; } /** Stores a collection of partition hashes for partition key {@code key}. */ - public Builder addPartitionHashes(PartitionKeyV2 key, Collection hashes) { - // The conflicts are used just as a hash container. Are not conflicts at the monnent. - hldr.hashConflicts.compute(key, (key0, hashes0) -> { + private Builder addPartitionHashes(PartitionKeyV2 key, Collection newHashes) { + if (partHashes == null) + partHashes = new HashMap<>(); + + partHashes.compute(key, (key0, hashes0) -> { if (hashes0 == null) hashes0 = new ArrayList<>(); - hashes0.addAll(hashes); + hashes0.addAll(newHashes); return hashes0; }); @@ -464,41 +480,47 @@ public Builder addPartitionHashes(PartitionKeyV2 key, Collection hashes) { - hashes.forEach((key, hash) -> addPartitionHashes(key, Collections.singletonList(hash))); + public void addPartitionHashes(Map newHashes) { + newHashes.forEach((key, hash) -> addPartitionHashes(key, Collections.singletonList(hash))); } - /** Stores a single partition hash. */ - public Builder addPartitionHash(PartitionKeyV2 key, PartitionHashRecordV2 hash) { - addPartitionHashes(key, Collections.singletonList(hash)); + /** Stores a single partition hash for partition key {@code key}. */ + public Builder addPartitionHash(PartitionKeyV2 key, PartitionHashRecordV2 newHash) { + addPartitionHashes(key, Collections.singletonList(newHash)); return this; } /** Sets partition hashes. */ public Builder partitionHashes(Map> partHashes) { - assert F.isEmpty(hldr.hashConflicts) : "Partitiin hashes are arleadt set."; + assert F.isEmpty(this.partHashes) : "Partition hashes are already set."; + assert partHashes != null; - // The conflicts are used just as a hash container. Are not conflicts at the monnent. - hldr.hashConflicts = partHashes; + this.partHashes = partHashes; return this; } /** Adds transaction conflicts. */ - public Builder addTxConflicts(List txConflicts) { - hldr.txHashConflicts.add(txConflicts); + public Builder addTxConflicts(List newTxConflicts) { + if (txHashConflicts == null) + txHashConflicts = new ArrayList<>(); + + txHashConflicts.add(newTxConflicts); return this; } /** Adds partially commited transactions. */ - public Builder addPartiallyCommited(ClusterNode node, Collection versions) { - hldr.partiallyCommittedTxs.compute(node, (node0, versions0) -> { + public Builder addPartiallyCommited(ClusterNode node, Collection newVerisons) { + if (partiallyCommittedTxs == null) + partiallyCommittedTxs = new HashMap<>(); + + partiallyCommittedTxs.compute(node, (node0, versions0) -> { if (versions0 == null) versions0 = new ArrayList<>(); - versions0.addAll(versions); + versions0.addAll(newVerisons); return versions0; }); @@ -508,7 +530,7 @@ public Builder addPartiallyCommited(ClusterNode node, Collection operationErrors ) { if (!operationErrors.isEmpty()) - return IdleVerifyResultV2.errors(operationErrors); + return IdleVerifyResultV2.builder().exceptions(operationErrors).build(); IdleVerifyResultV2.Builder bldr = IdleVerifyResultV2.builder(); From e5288245fb2b99bbad06f4900595d1d2394f0a23 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Tue, 11 Feb 2025 17:50:23 +0300 Subject: [PATCH 27/27] Renaming V2->V1 --- .../SnapshotCompressionBasicTest.java | 4 +-- ...ndHandlerCheckIncrementalSnapshotTest.java | 4 +-- .../ignite/util/GridCommandHandlerTest.java | 4 +-- .../management/cache/CacheFilterEnum.java | 2 +- .../cache/CacheIdleVerifyCommand.java | 6 ++-- .../management/cache/IdleVerifyException.java | 2 +- ...ifyResultV2.java => IdleVerifyResult.java} | 20 ++++++------ .../management/cache/IdleVerifyTaskV2.java | 8 ++--- .../cache/NoMatchingCachesException.java | 2 +- .../cache/VerifyBackupPartitionsDumpTask.java | 6 ++-- ...2.java => VerifyBackupPartitionsTask.java} | 10 +++--- .../snapshot/IgniteSnapshotManager.java | 8 ++--- .../snapshot/SnapshotCheckProcess.java | 6 ++-- .../persistence/snapshot/SnapshotChecker.java | 8 ++--- .../SnapshotPartitionsVerifyHandler.java | 6 ++-- .../SnapshotPartitionsVerifyResult.java | 12 +++---- ...idCacheFastNodeLeftForTransactionTest.java | 4 +-- .../HistoricalRebalanceCheckpointTest.java | 6 ++-- .../IgniteClusterSnapshotCheckTest.java | 32 +++++++++---------- .../IgniteClusterSnapshotRestoreSelfTest.java | 4 +-- .../IgniteSnapshotConsistencyTest.java | 4 +-- .../IgniteSnapshotRestoreFromRemoteTest.java | 4 +-- ...TxRollbackOnTimeoutOnePhaseCommitTest.java | 4 +-- ...thSmallTimeoutAndContentionOneKeyTest.java | 4 +-- .../junits/common/GridCommonAbstractTest.java | 8 ++--- ...teClusterSnapshotCheckWithIndexesTest.java | 8 ++--- 26 files changed, 93 insertions(+), 93 deletions(-) rename modules/core/src/main/java/org/apache/ignite/internal/management/cache/{IdleVerifyResultV2.java => IdleVerifyResult.java} (96%) rename modules/core/src/main/java/org/apache/ignite/internal/management/cache/{VerifyBackupPartitionsTaskV2.java => VerifyBackupPartitionsTask.java} (97%) diff --git a/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java b/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java index 08a4acbb1cdf2..0af75e51d29dd 100644 --- a/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java +++ b/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java @@ -49,7 +49,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridKernalContextImpl; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIO; import org.apache.ignite.internal.processors.compress.CompressionProcessor; @@ -336,7 +336,7 @@ protected void createTestSnapshot() throws Exception { for (String snpName : Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) { snp(ignite).createSnapshot(snpName, null, false, onlyPrimary).get(TIMEOUT); - IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr().checkSnapshot(snpName, null) + IdleVerifyResult res = ignite.context().cache().context().snapshotMgr().checkSnapshot(snpName, null) .get().idleVerifyResult(); StringBuilder b = new StringBuilder(); diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerCheckIncrementalSnapshotTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerCheckIncrementalSnapshotTest.java index cbee596138c41..21fe7d4091304 100644 --- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerCheckIncrementalSnapshotTest.java +++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerCheckIncrementalSnapshotTest.java @@ -29,7 +29,7 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.TestRecordingCommunicationSpi; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; import org.apache.ignite.internal.pagemem.wal.record.DataRecord; import org.apache.ignite.internal.pagemem.wal.record.TxRecord; @@ -378,7 +378,7 @@ public void atomicCachesAreSkippedDuringTheCheck() throws Exception { grid(0).snapshot().restoreSnapshot(atomicSnp, null, 1).get(); - IdleVerifyResultV2 idleVerRes = idleVerify(grid(0), CACHE, atomicCache); + IdleVerifyResult idleVerRes = idleVerify(grid(0), CACHE, atomicCache); idleVerRes.print(System.out::println, true); diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index ae6d68b2bb866..024cb3f154e28 100644 --- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -84,7 +84,7 @@ import org.apache.ignite.internal.client.util.GridConcurrentHashSet; import org.apache.ignite.internal.management.cache.FindAndDeleteGarbageInPersistenceTaskResult; import org.apache.ignite.internal.management.cache.IdleVerifyDumpTask; -import org.apache.ignite.internal.management.cache.VerifyBackupPartitionsTaskV2; +import org.apache.ignite.internal.management.cache.VerifyBackupPartitionsTask; import org.apache.ignite.internal.management.tx.TxInfo; import org.apache.ignite.internal.management.tx.TxTaskResult; import org.apache.ignite.internal.managers.communication.GridIoMessage; @@ -743,7 +743,7 @@ public void testIdleVerifyOnInactiveClusterWithPersistence() throws Exception { assertEquals(EXIT_CODE_UNEXPECTED_ERROR, execute("--cache", "idle_verify")); - assertContains(log, testOut.toString(), VerifyBackupPartitionsTaskV2.IDLE_VERIFY_ON_INACTIVE_CLUSTER_ERROR_MESSAGE); + assertContains(log, testOut.toString(), VerifyBackupPartitionsTask.IDLE_VERIFY_ON_INACTIVE_CLUSTER_ERROR_MESSAGE); assertContains(log, testOut.toString(), "Failed to perform operation"); srv.cluster().state(ACTIVE); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/CacheFilterEnum.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/CacheFilterEnum.java index 08ace4012f79d..db362918c65c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/CacheFilterEnum.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/CacheFilterEnum.java @@ -22,7 +22,7 @@ /** * Represents a type of cache(s) that can be used for comparing update counters and checksums between primary and backup partitions. *
- * @see VerifyBackupPartitionsTaskV2 + * @see VerifyBackupPartitionsTask */ public enum CacheFilterEnum { /** Default - user only, or all caches specified by name. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/CacheIdleVerifyCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/CacheIdleVerifyCommand.java index 867d37b1628fd..4dd5971729723 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/CacheIdleVerifyCommand.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/CacheIdleVerifyCommand.java @@ -36,8 +36,8 @@ /** Checks consistency of primary and backup partitions assuming no concurrent updates are happening in the cluster. */ @CliSubcommandsWithPrefix public class CacheIdleVerifyCommand - extends CommandRegistryImpl - implements ComputeCommand { + extends CommandRegistryImpl + implements ComputeCommand { /** */ public static final String IDLE_VERIFY_FILE_PREFIX = "idle_verify-"; @@ -72,7 +72,7 @@ public CacheIdleVerifyCommand() { } /** {@inheritDoc} */ - @Override public void printResult(CacheIdleVerifyCommandArg arg, IdleVerifyResultV2 res, Consumer printer) { + @Override public void printResult(CacheIdleVerifyCommandArg arg, IdleVerifyResult res, Consumer printer) { logParsedArgs(arg, printer); StringBuilder sb = new StringBuilder(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyException.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyException.java index 5f4b430b9a5de..e54fe3ad2a0b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyException.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyException.java @@ -23,7 +23,7 @@ import org.apache.ignite.internal.util.typedef.F; /** - * This exception is used to collect exceptions occured in {@link VerifyBackupPartitionsTaskV2} execution. + * This exception is used to collect exceptions occured in {@link VerifyBackupPartitionsTask} execution. */ public class IdleVerifyException extends IgniteException { /** */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResult.java similarity index 96% rename from modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java rename to modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResult.java index b2944e0a52548..0451a6ab60d38 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResultV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyResult.java @@ -45,9 +45,9 @@ import static org.apache.ignite.internal.util.IgniteUtils.nl; /** - * Encapsulates result of {@link VerifyBackupPartitionsTaskV2}. + * Encapsulates result of {@link VerifyBackupPartitionsTask}. */ -public class IdleVerifyResultV2 extends VisorDataTransferObject { +public class IdleVerifyResult extends VisorDataTransferObject { /** */ private static final long serialVersionUID = 0L; @@ -82,21 +82,21 @@ public class IdleVerifyResultV2 extends VisorDataTransferObject { /** * Default constructor for Externalizable. */ - public IdleVerifyResultV2() { + public IdleVerifyResult() { // No-op. } /** * @see Builder */ - private IdleVerifyResultV2(Map exceptions) { + private IdleVerifyResult(Map exceptions) { this.exceptions = exceptions; } /** * @see Builder */ - private IdleVerifyResultV2( + private IdleVerifyResult( Map> clusterHashes, @Nullable List> txHashConflicts, @Nullable Map> partiallyCommittedTxs, @@ -380,7 +380,7 @@ private void printConflicts(Consumer printer) { if (o == null || getClass() != o.getClass()) return false; - IdleVerifyResultV2 v2 = (IdleVerifyResultV2)o; + IdleVerifyResult v2 = (IdleVerifyResult)o; return Objects.equals(cntrConflicts, v2.cntrConflicts) && Objects.equals(hashConflicts, v2.hashConflicts) && Objects.equals(movingPartitions, v2.movingPartitions) && Objects.equals(lostPartitions, v2.lostPartitions) && @@ -402,7 +402,7 @@ private void printConflicts(Consumer printer) { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(IdleVerifyResultV2.class, this); + return S.toString(IdleVerifyResult.class, this); } /** @return A fresh result builder. */ @@ -410,7 +410,7 @@ public static Builder builder() { return new Builder(); } - /** Builder of {@link IdleVerifyResultV2}. Is not thread-safe. */ + /** Builder of {@link IdleVerifyResult}. Is not thread-safe. */ public static final class Builder { /** */ private @Nullable Map> partHashes; @@ -430,14 +430,14 @@ private Builder() { } /** Build the final result. */ - public IdleVerifyResultV2 build() { + public IdleVerifyResult build() { if (partHashes == null) partHashes = Collections.emptyMap(); if (exceptions == null) exceptions = Collections.emptyMap(); - return new IdleVerifyResultV2(partHashes, txHashConflicts, partiallyCommittedTxs, exceptions); + return new IdleVerifyResult(partHashes, txHashConflicts, partiallyCommittedTxs, exceptions); } /** Stores an exception if none is assigned for {@code node}. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyTaskV2.java index 1e03a864e0ecf..250757df90ea8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyTaskV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/IdleVerifyTaskV2.java @@ -26,15 +26,15 @@ * Task to verify checksums of backup partitions. */ @GridInternal -public class IdleVerifyTaskV2 extends VisorOneNodeTask { +public class IdleVerifyTaskV2 extends VisorOneNodeTask { /** */ private static final long serialVersionUID = 0L; /** {@inheritDoc} */ - @Override protected VisorJob job(CacheIdleVerifyCommandArg arg) { + @Override protected VisorJob job(CacheIdleVerifyCommandArg arg) { if (!ignite.context().state().publicApiActiveState(true)) - throw new IgniteException(VerifyBackupPartitionsTaskV2.IDLE_VERIFY_ON_INACTIVE_CLUSTER_ERROR_MESSAGE); + throw new IgniteException(VerifyBackupPartitionsTask.IDLE_VERIFY_ON_INACTIVE_CLUSTER_ERROR_MESSAGE); - return new IdleVerifyJob<>(arg, debug, VerifyBackupPartitionsTaskV2.class); + return new IdleVerifyJob<>(arg, debug, VerifyBackupPartitionsTask.class); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/NoMatchingCachesException.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/NoMatchingCachesException.java index 323c75992f441..4aa9bf269d737 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/NoMatchingCachesException.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/NoMatchingCachesException.java @@ -19,7 +19,7 @@ import org.apache.ignite.IgniteException; /** - * Runtime exception that can be thrown in {@link VerifyBackupPartitionsTaskV2} when no caches matching given + * Runtime exception that can be thrown in {@link VerifyBackupPartitionsTask} when no caches matching given * filter options can be found. */ public class NoMatchingCachesException extends IgniteException { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsDumpTask.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsDumpTask.java index 7ac73ad9e88e8..8e4711116d2d0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsDumpTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsDumpTask.java @@ -64,7 +64,7 @@ public class VerifyBackupPartitionsDumpTask extends ComputeTaskAdapter> partitions, - IdleVerifyResultV2 conflictRes, + IdleVerifyResult conflictRes, int skippedRecords ) throws IgniteException { String wd = ignite.configuration().getWorkDirectory(); @@ -204,7 +204,7 @@ private String writeHashes( /** */ private void writeResult( Map> partitions, - IdleVerifyResultV2 conflictRes, + IdleVerifyResult conflictRes, int skippedRecords, PrintWriter writer ) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTask.java similarity index 97% rename from modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java rename to modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTask.java index 0ac11085c5702..8777823695081 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTaskV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/cache/VerifyBackupPartitionsTask.java @@ -75,13 +75,13 @@ *
* Argument: Set of cache names, 'null' will trigger verification for all caches. *
- * Result: {@link IdleVerifyResultV2} with conflict partitions. + * Result: {@link IdleVerifyResult} with conflict partitions. *
* Works properly only on idle cluster - there may be false positive conflict reports if data in cluster is being * concurrently updated. */ @GridInternal -public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter { +public class VerifyBackupPartitionsTask extends ComputeTaskAdapter { /** First version of Ignite that is capable of executing Idle Verify V2. */ public static final IgniteProductVersion V2_SINCE_VER = IgniteProductVersion.fromString("2.5.3"); @@ -110,7 +110,7 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter results) throws IgniteException { + @Nullable @Override public IdleVerifyResult reduce(List results) throws IgniteException { return reduce0(results); } @@ -143,8 +143,8 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter results) { - IdleVerifyResultV2.Builder bldr = IdleVerifyResultV2.builder(); + public static IdleVerifyResult reduce0(List results) { + IdleVerifyResult.Builder bldr = IdleVerifyResult.builder(); for (ComputeJobResult res : results) { if (res.getException() != null) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java index f83ce0e595021..0e5225029edbd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java @@ -99,7 +99,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.cluster.DistributedConfigurationUtils; import org.apache.ignite.internal.events.DiscoveryCustomEvent; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.managers.communication.GridIoManager; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.communication.TransmissionCancelledException; @@ -1840,7 +1840,7 @@ public IgniteFuture cancelLocalRestoreTask(String name) { * @param name Snapshot name. * @param snpPath Snapshot directory path. * @return Future with the result of execution snapshot partitions verify task, which besides calculating partition - * hashes of {@link IdleVerifyResultV2} also contains the snapshot metadata distribution across the cluster. + * hashes of {@link IdleVerifyResult} also contains the snapshot metadata distribution across the cluster. */ public IgniteInternalFuture checkSnapshot(String name, @Nullable String snpPath) { return checkSnapshot(name, snpPath, -1); @@ -1853,7 +1853,7 @@ public IgniteInternalFuture checkSnapshot(String * @param snpPath Snapshot directory path. * @param incIdx Incremental snapshot index. * @return Future with the result of execution snapshot partitions verify task, which besides calculating partition - * hashes of {@link IdleVerifyResultV2} also contains the snapshot metadata distribution across the cluster. + * hashes of {@link IdleVerifyResult} also contains the snapshot metadata distribution across the cluster. */ public IgniteInternalFuture checkSnapshot(String name, @Nullable String snpPath, int incIdx) { A.notNullOrEmpty(name, "Snapshot name cannot be null or empty."); @@ -1884,7 +1884,7 @@ public IgniteInternalFuture checkSnapshot(String * @param incIdx Incremental snapshot index. * @param check If {@code true} check snapshot integrity. * @return Future with the result of execution snapshot partitions verify task, which besides calculating partition - * hashes of {@link IdleVerifyResultV2} also contains the snapshot metadata distribution across the cluster. + * hashes of {@link IdleVerifyResult} also contains the snapshot metadata distribution across the cluster. */ public IgniteInternalFuture checkSnapshot( String name, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java index a11ff41acb6f5..ca58e1126e350 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java @@ -41,7 +41,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.management.cache.PartitionKeyV2; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecordV2; import org.apache.ignite.internal.util.distributed.DistributedProcess; @@ -192,7 +192,7 @@ private void reduceIncrementalResults( errors.putIfAbsent(nodeId, asException(F.firstValue(incResp.exceptions()))); } - IdleVerifyResultV2 chkRes = checker.reduceIncrementalResults(reduced, mapErrors(errors)); + IdleVerifyResult chkRes = checker.reduceIncrementalResults(reduced, mapErrors(errors)); fut.onDone(new SnapshotPartitionsVerifyResult(clusterMetas, chkRes)); } @@ -252,7 +252,7 @@ private void reducePartitionsHashesResults( Map errors, GridFutureAdapter fut ) { - IdleVerifyResultV2.Builder bldr = IdleVerifyResultV2.builder(); + IdleVerifyResult.Builder bldr = IdleVerifyResult.builder(); Map errors0 = mapErrors(errors); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java index 871cbb52efd92..5a154182cd9b6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java @@ -49,7 +49,7 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.dump.DumpEntry; import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.management.cache.PartitionKeyV2; import org.apache.ignite.internal.managers.encryption.EncryptionCacheKeyProvider; import org.apache.ignite.internal.managers.encryption.GroupKey; @@ -574,14 +574,14 @@ private Map readTxCachesData(File snpDir) throws Ignit } /** */ - public IdleVerifyResultV2 reduceIncrementalResults( + public IdleVerifyResult reduceIncrementalResults( Map> results, Map operationErrors ) { if (!operationErrors.isEmpty()) - return IdleVerifyResultV2.builder().exceptions(operationErrors).build(); + return IdleVerifyResult.builder().exceptions(operationErrors).build(); - IdleVerifyResultV2.Builder bldr = IdleVerifyResultV2.builder(); + IdleVerifyResult.Builder bldr = IdleVerifyResult.builder(); Map> nodeTxHashMap = new HashMap<>(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java index 817866fb3f69f..706bcd34daf62 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java @@ -21,7 +21,7 @@ import java.util.Map; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.management.cache.PartitionKeyV2; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecordV2; @@ -59,7 +59,7 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext cctx) { /** {@inheritDoc} */ @Override public void complete(String name, Collection>> results) throws IgniteCheckedException { - IdleVerifyResultV2.Builder bldr = IdleVerifyResultV2.builder(); + IdleVerifyResult.Builder bldr = IdleVerifyResult.builder(); for (SnapshotHandlerResult> res : results) { if (res.error() != null) { @@ -72,7 +72,7 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext cctx) { bldr.addPartitionHash(entry.getKey(), entry.getValue()); } - IdleVerifyResultV2 verifyResult = bldr.build(); + IdleVerifyResult verifyResult = bldr.build(); if (!bldr.hasErrors() && !verifyResult.hasConflicts()) return; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyResult.java index 560f6237c1c67..e188758a1831f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyResult.java @@ -28,7 +28,7 @@ import java.util.stream.Collectors; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.dto.IgniteDataTransferObject; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.util.GridStringBuilder; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -36,7 +36,7 @@ /** * The result of execution snapshot partitions verify task which besides calculating partition hashes of - * {@link IdleVerifyResultV2} also contains the snapshot metadata distribution across the cluster. + * {@link IdleVerifyResult} also contains the snapshot metadata distribution across the cluster. */ public class SnapshotPartitionsVerifyResult extends IgniteDataTransferObject { /** Serial version uid. */ @@ -46,7 +46,7 @@ public class SnapshotPartitionsVerifyResult extends IgniteDataTransferObject { private Map> metas; /** Result of cluster nodes partitions comparison. */ - @Nullable private IdleVerifyResultV2 idleRes; + @Nullable private IdleVerifyResult idleRes; /** Default constructor. */ public SnapshotPartitionsVerifyResult() { @@ -59,7 +59,7 @@ public SnapshotPartitionsVerifyResult() { */ SnapshotPartitionsVerifyResult( Map> metas, - @Nullable IdleVerifyResultV2 idleRes + @Nullable IdleVerifyResult idleRes ) { this.metas = metas; this.idleRes = idleRes; @@ -103,7 +103,7 @@ public void print(Consumer printer) { /** * @return Result of cluster nodes partitions comparison. */ - public IdleVerifyResultV2 idleVerifyResult() { + public IdleVerifyResult idleVerifyResult() { return idleRes; } @@ -123,6 +123,6 @@ public Map exceptions() { /** {@inheritDoc} */ @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { metas = U.readMap(in); - idleRes = (IdleVerifyResultV2)in.readObject(); + idleRes = (IdleVerifyResult)in.readObject(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFastNodeLeftForTransactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFastNodeLeftForTransactionTest.java index c394d4f59bd90..55cf4cd1ba8bc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFastNodeLeftForTransactionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFastNodeLeftForTransactionTest.java @@ -30,7 +30,7 @@ import org.apache.ignite.configuration.ConnectorConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishFuture; import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyImpl; import org.apache.ignite.internal.util.typedef.internal.SB; @@ -236,7 +236,7 @@ private void check( checkCacheData(cacheValues, cacheName); - IdleVerifyResultV2 idleVerifyResV2 = idleVerify(stoppedNode, null); + IdleVerifyResult idleVerifyResV2 = idleVerify(stoppedNode, null); SB sb = new SB(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/HistoricalRebalanceCheckpointTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/HistoricalRebalanceCheckpointTest.java index 52e2b39cc92b6..16f764d9cfc61 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/HistoricalRebalanceCheckpointTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/HistoricalRebalanceCheckpointTest.java @@ -44,7 +44,7 @@ import org.apache.ignite.failure.StopNodeFailureHandler; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.TestRecordingCommunicationSpi; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; 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; @@ -191,7 +191,7 @@ public void testDelayed1PhaseCommitResponses() throws Exception { // Storing the highest counters on backup. forceCheckpoint(); - IdleVerifyResultV2 checkRes = idleVerify(prim, DEFAULT_CACHE_NAME); + IdleVerifyResult checkRes = idleVerify(prim, DEFAULT_CACHE_NAME); Map conflicts = F.flatCollections(checkRes.counterConflicts().values()) .stream().collect(Collectors.toMap(PartitionHashRecordV2::isPrimary, Functions.identity())); @@ -394,7 +394,7 @@ private void doTestDelayedToBackupsRequests(int nodes, boolean putAfterGaps) thr rebalanceLsnr.check(); - IdleVerifyResultV2 checkRes = idleVerify(prim, DEFAULT_CACHE_NAME); + IdleVerifyResult checkRes = idleVerify(prim, DEFAULT_CACHE_NAME); assertFalse(checkRes.hasConflicts()); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java index 55eb57019f07e..7364c21a3a811 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java @@ -61,9 +61,9 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.management.cache.CacheFilterEnum; import org.apache.ignite.internal.management.cache.CacheIdleVerifyCommandArg; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.management.cache.PartitionKeyV2; -import org.apache.ignite.internal.management.cache.VerifyBackupPartitionsTaskV2; +import org.apache.ignite.internal.management.cache.VerifyBackupPartitionsTask; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupContext; @@ -145,7 +145,7 @@ public void testClusterSnapshotCheck() throws Exception { createAndCheckSnapshot(ignite, SNAPSHOT_NAME); - IdleVerifyResultV2 res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); + IdleVerifyResult res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); res.print(b::append, true); @@ -169,7 +169,7 @@ public void testClusterSnapshotCheckMissedPart() throws Exception { assertTrue(part0.toString(), part0.toFile().exists()); assertTrue(part0.toFile().delete()); - IdleVerifyResultV2 res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); + IdleVerifyResult res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); res.print(b::append, true); @@ -193,7 +193,7 @@ public void testClusterSnapshotCheckMissedGroup() throws Exception { assertTrue(dir.toString(), dir.toFile().exists()); assertTrue(U.delete(dir)); - IdleVerifyResultV2 res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); + IdleVerifyResult res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); res.print(b::append, true); @@ -236,7 +236,7 @@ public void testClusterSnapshotCheckWithNodeFilter() throws Exception { createAndCheckSnapshot(ig0, SNAPSHOT_NAME); - IdleVerifyResultV2 res = snp(ig0).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); + IdleVerifyResult res = snp(ig0).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); res.print(b::append, true); @@ -312,7 +312,7 @@ public void testClusterSnapshotCheckPartitionCounters() throws Exception { pageStore.finishRecover(); } - IdleVerifyResultV2 res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); + IdleVerifyResult res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); res.print(b::append, true); @@ -362,7 +362,7 @@ public void testClusterSnapshotCheckOtherCluster() throws Exception { ignite.cluster().baselineAutoAdjustEnabled(false); ignite.cluster().state(ACTIVE); - IdleVerifyResultV2 res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); + IdleVerifyResult res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); res.print(b::append, true); @@ -386,7 +386,7 @@ public void testClusterSnapshotCheckCRCFail() throws Exception { corruptPartitionFile(ignite, SNAPSHOT_NAME, dfltCacheCfg, PART_ID); - IdleVerifyResultV2 res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null, null, false, -1, false).get().idleVerifyResult(); + IdleVerifyResult res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null, null, false, -1, false).get().idleVerifyResult(); assertEquals("Check must be disabled", 0, res.exceptions().size()); @@ -494,7 +494,7 @@ public void testClusterSnapshotCheckFailsOnPartitionDataDiffers() throws Excepti assertNotNull(part0); assertTrue(part0.toString(), part0.toFile().exists()); - IdleVerifyResultV2 res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); + IdleVerifyResult res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); res.print(b::append, true); @@ -521,9 +521,9 @@ public void testClusterSnapshotCheckHashesSameAsIdleVerifyHashes() throws Except arg.cacheFilter(CacheFilterEnum.USER); arg.checkCrc(true); - IdleVerifyResultV2 idleVerifyRes = ignite.compute().execute(new TestVisorBackupPartitionsTask(), arg); + IdleVerifyResult idleVerifyRes = ignite.compute().execute(new TestVisorBackupPartitionsTask(), arg); - IdleVerifyResultV2 snpVerifyRes = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); + IdleVerifyResult snpVerifyRes = snp(ignite).checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); Map> idleVerifyHashes = jobResults.get(TestVisorBackupPartitionsTask.class); Map> snpCheckHashes = jobResults.get(TestVisorBackupPartitionsTask.class); @@ -571,7 +571,7 @@ public void testCheckFromLesserTopology() throws Exception { assumeFalse(encryption); int srvCnt = 3; - IdleVerifyResultV2 chkRes; + IdleVerifyResult chkRes; IgniteEx client = startGridsWithSnapshot(srvCnt, CACHE_KEYS_RANGE, true, true); @@ -1447,10 +1447,10 @@ private static void corruptPartitionFile( } /** */ - private class TestVisorBackupPartitionsTask extends VerifyBackupPartitionsTaskV2 { + private class TestVisorBackupPartitionsTask extends VerifyBackupPartitionsTask { /** {@inheritDoc} */ - @Override public @Nullable IdleVerifyResultV2 reduce(List results) throws IgniteException { - IdleVerifyResultV2 res = super.reduce(results); + @Override public @Nullable IdleVerifyResult reduce(List results) throws IgniteException { + IdleVerifyResult res = super.reduce(results); saveHashes(TestVisorBackupPartitionsTask.class, results); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java index 335078fc7abf8..ac7c42dcd39c6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java @@ -51,7 +51,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.TestRecordingCommunicationSpi; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; @@ -142,7 +142,7 @@ public void testClusterSnapshotRestoreFromCustomDir() throws Exception { createAndCheckSnapshot(ignite, SNAPSHOT_NAME, snpDir.toString(), TIMEOUT); // Check snapshot. - IdleVerifyResultV2 res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, snpDir.getAbsolutePath()).get(TIMEOUT) + IdleVerifyResult res = snp(ignite).checkSnapshot(SNAPSHOT_NAME, snpDir.getAbsolutePath()).get(TIMEOUT) .idleVerifyResult(); StringBuilder sb = new StringBuilder(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java index 370b92cdd07fd..a573e4079a2f7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java @@ -37,7 +37,7 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.TestRecordingCommunicationSpi; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheMessage; @@ -210,7 +210,7 @@ public void testConcurrentPutWithStaleTopologyVersion() throws Exception { snpFut.get(getTestTimeout()); putFut.get(getTestTimeout()); - IdleVerifyResultV2 snpVerifyRes = crd.context().cache().context().snapshotMgr() + IdleVerifyResult snpVerifyRes = crd.context().cache().context().snapshotMgr() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); assertFalse(snpVerifyRes.hasConflicts()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java index cbb5767e9c45c..7451e46d054af 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java @@ -45,7 +45,7 @@ import org.apache.ignite.failure.StopNodeOrHaltFailureHandler; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.TestRecordingCommunicationSpi; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage; import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; import org.apache.ignite.internal.util.typedef.F; @@ -246,7 +246,7 @@ public void testRestoreFromAnEmptyNode() throws Exception { awaitPartitionMapExchange(); // Ensure that the snapshot check command succeeds. - IdleVerifyResultV2 res = emptyNode.context().cache().context().snapshotMgr() + IdleVerifyResult res = emptyNode.context().cache().context().snapshotMgr() .checkSnapshot(SNAPSHOT_NAME, null).get(TIMEOUT).idleVerifyResult(); StringBuilder buf = new StringBuilder(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutOnePhaseCommitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutOnePhaseCommitTest.java index c0c24bd969f69..38a3f5c2af255 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutOnePhaseCommitTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutOnePhaseCommitTest.java @@ -27,7 +27,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.TestRecordingCommunicationSpi; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; @@ -193,7 +193,7 @@ private void doTestRollbackOnTimeoutPartitionDesync(TransactionConcurrency concu fut.get(); - IdleVerifyResultV2 res = idleVerify(client, DEFAULT_CACHE_NAME); + IdleVerifyResult res = idleVerify(client, DEFAULT_CACHE_NAME); assertPartitionsSame(res); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxWithSmallTimeoutAndContentionOneKeyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxWithSmallTimeoutAndContentionOneKeyTest.java index 29a1ef76ff210..ff2e4099b7a56 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxWithSmallTimeoutAndContentionOneKeyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxWithSmallTimeoutAndContentionOneKeyTest.java @@ -34,7 +34,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.management.cache.PartitionKeyV2; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecordV2; import org.apache.ignite.internal.util.typedef.internal.SB; @@ -202,7 +202,7 @@ public void test() throws Exception { f.get(); - IdleVerifyResultV2 idleVerifyResult = idleVerify(igClient, DEFAULT_CACHE_NAME); + IdleVerifyResult idleVerifyResult = idleVerify(igClient, DEFAULT_CACHE_NAME); log.info("Current counter value:" + cnt.get()); diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java index 3e909de461028..122cec2361dd2 100755 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java @@ -81,7 +81,7 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.management.cache.CacheIdleVerifyCommandArg; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.management.cache.IdleVerifyTaskV2; import org.apache.ignite.internal.pagemem.wal.record.DataRecord; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -2224,7 +2224,7 @@ protected void forceCheckpoint(Collection nodes, String reason) throws I * @return Conflicts result. * @throws IgniteException If none caches or node found. */ - protected IdleVerifyResultV2 idleVerify(Ignite ig, @Nullable String... caches) throws Exception { + protected IdleVerifyResult idleVerify(Ignite ig, @Nullable String... caches) throws Exception { log.info("Starting idleVerify ..."); IgniteEx ig0 = (IgniteEx)ig; @@ -2248,7 +2248,7 @@ protected IdleVerifyResultV2 idleVerify(Ignite ig, @Nullable String... caches) t taskArg.caches(cacheNames.toArray(U.EMPTY_STRS)); - return ((VisorTaskResult)ig.compute().execute( + return ((VisorTaskResult)ig.compute().execute( IdleVerifyTaskV2.class.getName(), new VisorTaskArgument<>(node.id(), taskArg, false) )).result(); @@ -2447,7 +2447,7 @@ public static void awaitCacheOnClient(Ignite client, String cacheName) { /** * @param res Response. */ - protected static void assertPartitionsSame(IdleVerifyResultV2 res) throws AssertionFailedError { + protected static void assertPartitionsSame(IdleVerifyResult res) throws AssertionFailedError { if (res.hasConflicts()) { StringBuilder b = new StringBuilder(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java index 39c6926a230ee..c7f1316077635 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java @@ -24,7 +24,7 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.management.cache.IdleVerifyResultV2; +import org.apache.ignite.internal.management.cache.IdleVerifyResult; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.lang.IgnitePredicate; import org.junit.Test; @@ -44,7 +44,7 @@ public void testClusterSnapshotCheckEmptyCache() throws Exception { snp(ignite).createSnapshot(SNAPSHOT_NAME, null, false, onlyPrimary).get(TIMEOUT); - IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr() + IdleVerifyResult res = ignite.context().cache().context().snapshotMgr() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); @@ -62,7 +62,7 @@ public void testClusterSnapshotCheckWithIndexes() throws Exception { createAndCheckSnapshot(ignite, SNAPSHOT_NAME, null, TIMEOUT); - IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr() + IdleVerifyResult res = ignite.context().cache().context().snapshotMgr() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); @@ -89,7 +89,7 @@ public void testClusterSnapshotCheckWithNodeFilter() throws Exception { createAndCheckSnapshot(grid(0), SNAPSHOT_NAME, null, TIMEOUT); - IdleVerifyResultV2 res = grid(0).context().cache().context().snapshotMgr() + IdleVerifyResult res = grid(0).context().cache().context().snapshotMgr() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder();