timoninmaxim commented on code in PR #11391:
URL: https://github.com/apache/ignite/pull/11391#discussion_r1726789239
##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotChecker.java:
##########
@@ -536,24 +618,145 @@ private boolean hasExpiringEntries(
return rootIO.getCount(pageAddr) != 0;
}
+ /** Launches local partitions checking and waits for the result, handles
execution exceptions. */
+ public Map<PartitionKeyV2, PartitionHashRecordV2> checkPartitionsResult(
+ SnapshotMetadata meta,
+ File snpDir,
+ @Nullable Collection<String> groups,
+ boolean forCreation,
+ boolean checkParts,
+ boolean skipPartsHashes
+ ) {
+ try {
+ return checkPartitions(meta, snpDir, groups, forCreation,
checkParts, skipPartsHashes).get();
Review Comment:
Delegate `get()` call to an invoker. Checker should have only one method
that checks partitions.
##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * 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.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.internal.GridKernalContext;
+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.PartitionKeyV2;
+import
org.apache.ignite.internal.processors.cache.verify.PartitionHashRecordV2;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
+import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
+import static
org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CHECK_SNAPSHOT_METAS;
+import static
org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CHECK_SNAPSHOT_PARTS;
+
+/** Distributed process of snapshot checking (with the partition hashes). */
+public class SnapshotCheckProcess {
+ /** */
+ private final IgniteLogger log;
+
+ /** */
+ private final GridKernalContext kctx;
+
+ /** Operation contexts by name. */
+ private final Map<String, SnapshotCheckContext> contexts = new
ConcurrentHashMap<>();
+
+ /** Cluster-wide operation futures per snapshot called from current node.
*/
+ private final Map<UUID,
GridFutureAdapter<SnapshotPartitionsVerifyTaskResult>> clusterOpFuts = new
ConcurrentHashMap<>();
+
+ /** Check metas first phase subprocess. */
+ private final DistributedProcess<SnapshotCheckProcessRequest,
SnapshotCheckResponse> phase1CheckMetas;
+
+ /** Partition hashes second phase subprocess. */
+ private final DistributedProcess<SnapshotCheckProcessRequest,
SnapshotCheckResponse> phase2PartsHashes;
+
+ /** */
+ public SnapshotCheckProcess(GridKernalContext kctx) {
+ this.kctx = kctx;
+
+ log = kctx.log(getClass());
+
+ phase1CheckMetas = new DistributedProcess<>(kctx,
CHECK_SNAPSHOT_METAS, this::prepareAndCheckMetas,
+ this::reducePreparationAndMetasCheck);
+
+ phase2PartsHashes = new DistributedProcess<>(kctx,
CHECK_SNAPSHOT_PARTS, this::validateParts,
+ this::reduceValidatePartsAndFinish);
+
+ kctx.event().addLocalEventListener(evt -> {
+ UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
+
+ Throwable err = new ClusterTopologyCheckedException("Snapshot
validation stopped. A required node left " +
+ "the cluster [nodeId=" + nodeId + ']');
+
+ contexts.values().forEach(ctx -> {
+ if (ctx.req.nodes().contains(nodeId)) {
+ ctx.locProcFut.onDone(err);
+
+ // We have no a guaranty that a node-left-event is
processed strictly before the 1st phase reduce which
Review Comment:
guarantee
##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * 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.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.internal.GridKernalContext;
+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.PartitionKeyV2;
+import
org.apache.ignite.internal.processors.cache.verify.PartitionHashRecordV2;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
+import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
+import static
org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CHECK_SNAPSHOT_METAS;
+import static
org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CHECK_SNAPSHOT_PARTS;
+
+/** Distributed process of snapshot checking (with the partition hashes). */
+public class SnapshotCheckProcess {
+ /** */
+ private final IgniteLogger log;
+
+ /** */
+ private final GridKernalContext kctx;
+
+ /** Operation contexts by name. */
+ private final Map<String, SnapshotCheckContext> contexts = new
ConcurrentHashMap<>();
+
+ /** Cluster-wide operation futures per snapshot called from current node.
*/
+ private final Map<UUID,
GridFutureAdapter<SnapshotPartitionsVerifyTaskResult>> clusterOpFuts = new
ConcurrentHashMap<>();
+
+ /** Check metas first phase subprocess. */
+ private final DistributedProcess<SnapshotCheckProcessRequest,
SnapshotCheckResponse> phase1CheckMetas;
+
+ /** Partition hashes second phase subprocess. */
+ private final DistributedProcess<SnapshotCheckProcessRequest,
SnapshotCheckResponse> phase2PartsHashes;
+
+ /** */
+ public SnapshotCheckProcess(GridKernalContext kctx) {
+ this.kctx = kctx;
+
+ log = kctx.log(getClass());
+
+ phase1CheckMetas = new DistributedProcess<>(kctx,
CHECK_SNAPSHOT_METAS, this::prepareAndCheckMetas,
+ this::reducePreparationAndMetasCheck);
+
+ phase2PartsHashes = new DistributedProcess<>(kctx,
CHECK_SNAPSHOT_PARTS, this::validateParts,
+ this::reduceValidatePartsAndFinish);
+
+ kctx.event().addLocalEventListener(evt -> {
+ UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
+
+ Throwable err = new ClusterTopologyCheckedException("Snapshot
validation stopped. A required node left " +
+ "the cluster [nodeId=" + nodeId + ']');
+
+ contexts.values().forEach(ctx -> {
+ if (ctx.req.nodes().contains(nodeId)) {
+ ctx.locProcFut.onDone(err);
+
+ // We have no a guaranty that a node-left-event is
processed strictly before the 1st phase reduce which
+ // can handle this error.
+ GridFutureAdapter<?> clusterFut =
clusterOpFuts.get(ctx.req.requestId());
+
+ if (clusterFut != null)
+ clusterFut.onDone(err);
+ }
+ });
+ }, EVT_NODE_FAILED, EVT_NODE_LEFT);
+ }
+
+ /** */
+ Map<String, SnapshotCheckContext> contexts() {
Review Comment:
This method isn't used
##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcess.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * 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.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.internal.GridKernalContext;
+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.PartitionKeyV2;
+import
org.apache.ignite.internal.processors.cache.verify.PartitionHashRecordV2;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
+import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
+import static
org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CHECK_SNAPSHOT_METAS;
+import static
org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CHECK_SNAPSHOT_PARTS;
+
+/** Distributed process of snapshot checking (with the partition hashes). */
+public class SnapshotCheckProcess {
+ /** */
+ private final IgniteLogger log;
+
+ /** */
+ private final GridKernalContext kctx;
+
+ /** Operation contexts by name. */
+ private final Map<String, SnapshotCheckContext> contexts = new
ConcurrentHashMap<>();
+
+ /** Cluster-wide operation futures per snapshot called from current node.
*/
+ private final Map<UUID,
GridFutureAdapter<SnapshotPartitionsVerifyTaskResult>> clusterOpFuts = new
ConcurrentHashMap<>();
+
+ /** Check metas first phase subprocess. */
+ private final DistributedProcess<SnapshotCheckProcessRequest,
SnapshotCheckResponse> phase1CheckMetas;
+
+ /** Partition hashes second phase subprocess. */
+ private final DistributedProcess<SnapshotCheckProcessRequest,
SnapshotCheckResponse> phase2PartsHashes;
+
+ /** */
+ public SnapshotCheckProcess(GridKernalContext kctx) {
+ this.kctx = kctx;
+
+ log = kctx.log(getClass());
+
+ phase1CheckMetas = new DistributedProcess<>(kctx,
CHECK_SNAPSHOT_METAS, this::prepareAndCheckMetas,
+ this::reducePreparationAndMetasCheck);
+
+ phase2PartsHashes = new DistributedProcess<>(kctx,
CHECK_SNAPSHOT_PARTS, this::validateParts,
+ this::reduceValidatePartsAndFinish);
+
+ kctx.event().addLocalEventListener(evt -> {
+ UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
+
+ Throwable err = new ClusterTopologyCheckedException("Snapshot
validation stopped. A required node left " +
+ "the cluster [nodeId=" + nodeId + ']');
+
+ contexts.values().forEach(ctx -> {
+ if (ctx.req.nodes().contains(nodeId)) {
+ ctx.locProcFut.onDone(err);
+
+ // We have no a guaranty that a node-left-event is
processed strictly before the 1st phase reduce which
+ // can handle this error.
+ GridFutureAdapter<?> clusterFut =
clusterOpFuts.get(ctx.req.requestId());
+
+ if (clusterFut != null)
+ clusterFut.onDone(err);
+ }
+ });
+ }, EVT_NODE_FAILED, EVT_NODE_LEFT);
+ }
+
+ /** */
+ Map<String, SnapshotCheckContext> contexts() {
+ return Collections.unmodifiableMap(contexts);
+ }
+
+ /**
+ * Stops all the processes with the passed exception.
+ *
+ * @param err The interrupt reason.
+ */
+ void interrupt(Throwable err) {
+ contexts.forEach((snpName, ctx) -> ctx.locProcFut.onDone(err));
+
+ contexts.clear();
+
+ clusterOpFuts.forEach((reqId, fut) -> fut.onDone(err));
+ }
+
+ /** Phase 2 and process finish. */
+ private IgniteInternalFuture<?> reduceValidatePartsAndFinish(
+ UUID reqId,
+ Map<UUID, SnapshotCheckResponse> results,
+ Map<UUID, Throwable> errors
+ ) {
+ SnapshotCheckContext ctx = context(null, reqId);
+
+ if (ctx == null)
+ return new GridFinishedFuture<>();
+
+ contexts.remove(ctx.req.snapshotName());
+
+ if (log.isInfoEnabled())
+ log.info("Finished snapshot validation [req=" + ctx.req + ']');
+
+ GridFutureAdapter<SnapshotPartitionsVerifyTaskResult> clusterOpFut =
clusterOpFuts.get(reqId);
+
+ if (clusterOpFut == null)
+ return new GridFinishedFuture<>();
+
+ assert results.values().stream().noneMatch(res -> res != null &&
res.metas != null);
+
+ if (ctx.req.allRestoreHandlers()) {
+ try {
+ if (!errors.isEmpty())
+ throw F.firstValue(errors);
+
+ Map<ClusterNode, Map<String, SnapshotHandlerResult<?>>> cstRes
= mapCustomHandlersResults(results, ctx.req.nodes());
+
+
kctx.cache().context().snapshotMgr().checker().checkCustomHandlersResults(ctx.req.snapshotName(),
cstRes);
+
+ clusterOpFut.onDone(new
SnapshotPartitionsVerifyTaskResult(ctx.clusterMetas, null));
+ }
+ catch (Throwable err) {
+ clusterOpFut.onDone(err);
+ }
+ }
+ else {
+ Map<ClusterNode, Exception> errors0 = mapErrors(errors);
+
+ if (!results.isEmpty()) {
+ Map<ClusterNode, Map<PartitionKeyV2, PartitionHashRecordV2>>
results0 = mapPartsHashes(results, ctx.req.nodes());
+
+ IdleVerifyResultV2 chkRes =
SnapshotChecker.reduceHashesResults(results0, errors0);
+
+ clusterOpFut.onDone(new
SnapshotPartitionsVerifyTaskResult(ctx.clusterMetas, chkRes));
+ }
+ else
+ clusterOpFut.onDone(new
IgniteSnapshotVerifyException(errors0));
+ }
+
+ return new GridFinishedFuture<>();
+ }
+
+ /** Phase 2 beginning. */
+ private IgniteInternalFuture<SnapshotCheckResponse>
validateParts(SnapshotCheckProcessRequest req) {
+ if (!req.nodes().contains(kctx.localNodeId()))
+ return new GridFinishedFuture<>();
+
+ SnapshotCheckContext ctx = context(req.snapshotName(),
req.requestId());
+
+ assert ctx != null;
+
+ if (ctx.locMeta == null)
+ return new GridFinishedFuture<>();
+
+ IgniteSnapshotManager snpMgr = kctx.cache().context().snapshotMgr();
+
+ GridFutureAdapter<SnapshotCheckResponse> phaseFut = ctx.phaseFut();
+
+ // Might be already finished by asynchronous leave of a required node.
+ if (!phaseFut.isDone()) {
+ CompletableFuture<? extends Map<?, ?>> 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);
+
+ workingFut.whenComplete((res, err) -> {
+ if (err != null)
+ phaseFut.onDone(err);
+ else
+ phaseFut.onDone(new SnapshotCheckResponse(res));
+ });
+ }
+
+ return phaseFut;
+ }
+
+ /** */
+ private Map<ClusterNode, Exception> mapErrors(Map<UUID, Throwable> 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())));
+ }
+
+ /** */
+ private Map<ClusterNode, Map<PartitionKeyV2, PartitionHashRecordV2>>
mapPartsHashes(
+ Map<UUID, SnapshotCheckResponse> results,
+ Collection<UUID> requiredNodes
+ ) {
+ return results.entrySet().stream()
+ .filter(e -> requiredNodes.contains(e.getKey()) && e.getValue() !=
null)
+ .collect(Collectors.toMap(e ->
kctx.cluster().get().node(e.getKey()), e -> e.getValue().partsHashes()));
+ }
+
+ /** */
+ private Map<ClusterNode, Map<String, SnapshotHandlerResult<?>>>
mapCustomHandlersResults(
+ Map<UUID, SnapshotCheckResponse> results,
+ Set<UUID> requiredNodes
+ ) {
+ return results.entrySet().stream()
+ .filter(e -> requiredNodes.contains(e.getKey()) && e.getValue() !=
null)
+ .collect(Collectors.toMap(e ->
kctx.cluster().get().node(e.getKey()), e ->
e.getValue().customHandlersResults()));
+ }
+
+ /**
+ * @param snpName Snapshot name of the validation process. If {@code
null}, ignored.
+ * @param reqId If {@code snpName} is {@code null}, is used to find the
operation request.
+ * @return Current snapshot checking context by {@code snpName} or {@code
reqId}.
+ */
+ private @Nullable SnapshotCheckContext context(@Nullable String snpName,
UUID reqId) {
+ SnapshotCheckContext ctx = snpName == null
+ ? contexts.values().stream().filter(ctx0 ->
ctx0.req.requestId().equals(reqId)).findFirst().orElse(null)
+ : contexts.get(snpName);
+
+ assert ctx == null || ctx.req.requestId().equals(reqId);
+
+ return ctx;
+ }
+
+ /** Phase 1 beginning: prepare, collect and check local metas. */
+ private IgniteInternalFuture<SnapshotCheckResponse>
prepareAndCheckMetas(SnapshotCheckProcessRequest req) {
+ if (!req.nodes().contains(kctx.localNodeId()))
+ return new GridFinishedFuture<>();
+
+ if (kctx.isStopping())
+ return new GridFinishedFuture<>(new NodeStoppingException("The
node is stopping: " + kctx.localNodeId()));
+
+ SnapshotCheckContext ctx =
contexts.computeIfAbsent(req.snapshotName(), snpName -> new
SnapshotCheckContext(req));
+
+ if (!ctx.req.requestId().equals(req.requestId())) {
+ return new GridFinishedFuture<>(new
IllegalStateException("Validation of snapshot '" + req.snapshotName()
Review Comment:
Just throw exception here
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]